From 1c20b8397299b17279e30ed4dca1f9efe6b8d9ec Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 20 Sep 2019 21:51:05 +0800 Subject: [PATCH 001/268] [FLINK-14096][client] Merge NewClusterClient into ClusterClient * [FLINK-14096][client] Merge NewClusterClient into ClusterClient * [FLINK-14096][client] Reduce unnecessary cast Merge in a squashed commit referred to @azagrebin's comment that easy to blame git history --- .../flink/client/program/ClusterClient.java | 18 +++++++ .../client/program/MiniClusterClient.java | 2 +- .../client/program/NewClusterClient.java | 50 ------------------- .../program/rest/RestClusterClient.java | 3 +- .../client/gateway/local/ProgramDeployer.java | 3 +- .../org/apache/flink/yarn/YARNITCase.java | 24 +++------ .../flink/yarn/util/FakeClusterClient.java | 19 +++++++ 7 files changed, 46 insertions(+), 73 deletions(-) delete mode 100644 flink-clients/src/main/java/org/apache/flink/client/program/NewClusterClient.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index f75f04423d..0ec54ccb8d 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -38,6 +38,7 @@ import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.util.FlinkException; import org.apache.flink.util.OptionalFailure; @@ -46,6 +47,7 @@ import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.net.URISyntaxException; @@ -402,6 +404,22 @@ public abstract class ClusterClient implements AutoCloseable { */ public abstract JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException; + /** + * Submit the given {@link JobGraph} to the cluster. + * + * @param jobGraph to submit + * @return Future which is completed with the {@link JobSubmissionResult} + */ + public abstract CompletableFuture submitJob(@Nonnull JobGraph jobGraph); + + /** + * Request the {@link JobResult} for the given {@link JobID}. + * + * @param jobId for which to request the {@link JobResult} + * @return Future which is completed with the {@link JobResult} + */ + public abstract CompletableFuture requestJobResult(@Nonnull JobID jobId); + public void shutDownCluster() { throw new UnsupportedOperationException("The " + getClass().getSimpleName() + " does not support shutDownCluster."); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index d983551e47..5b0214a8f2 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -46,7 +46,7 @@ import java.util.concurrent.ExecutionException; /** * Client to interact with a {@link MiniCluster}. */ -public class MiniClusterClient extends ClusterClient implements NewClusterClient { +public class MiniClusterClient extends ClusterClient { private final MiniCluster miniCluster; diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/NewClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/NewClusterClient.java deleted file mode 100644 index 513f7da5f0..0000000000 --- a/flink-clients/src/main/java/org/apache/flink/client/program/NewClusterClient.java +++ /dev/null @@ -1,50 +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.client.program; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobSubmissionResult; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmaster.JobResult; - -import javax.annotation.Nonnull; - -import java.util.concurrent.CompletableFuture; - -/** - * Interface for the new cluster client. - */ -public interface NewClusterClient { - - /** - * Submit the given {@link JobGraph} to the cluster. - * - * @param jobGraph to submit - * @return Future which is completed with the {@link JobSubmissionResult} - */ - CompletableFuture submitJob(@Nonnull JobGraph jobGraph); - - /** - * Request the {@link JobResult} for the given {@link JobID}. - * - * @param jobId for which to request the {@link JobResult} - * @return Future which is completed with the {@link JobResult} - */ - CompletableFuture requestJobResult(@Nonnull JobID jobId); -} diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index c1e88b53b1..0303620e77 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -27,7 +27,6 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.DetachedJobExecutionResult; -import org.apache.flink.client.program.NewClusterClient; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.client.program.rest.retry.ExponentialWaitStrategy; import org.apache.flink.client.program.rest.retry.WaitStrategy; @@ -128,7 +127,7 @@ import java.util.stream.Collectors; /** * A {@link ClusterClient} implementation that communicates via HTTP REST requests. */ -public class RestClusterClient extends ClusterClient implements NewClusterClient { +public class RestClusterClient extends ClusterClient { private final RestClusterClientConfiguration restClusterClientConfiguration; diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java index b484521edd..61de25c54d 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java @@ -21,7 +21,6 @@ package org.apache.flink.table.client.gateway.local; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.table.client.gateway.SqlExecutionException; import org.apache.flink.table.client.gateway.local.result.Result; @@ -122,7 +121,7 @@ public class ProgramDeployer implements Runnable { // get result if (awaitJobResult) { // we need to hard cast for now - final JobExecutionResult jobResult = ((RestClusterClient) clusterClient) + final JobExecutionResult jobResult = clusterClient .requestJobResult(jobGraph.getJobID()) .get() .toJobExecutionResult(context.getClassLoader()); // throws exception if job fails diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java index 82d565740e..e894f654a5 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java @@ -21,7 +21,6 @@ package org.apache.flink.yarn; import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -44,7 +43,6 @@ import java.time.Duration; import java.util.Arrays; import java.util.concurrent.CompletableFuture; -import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.junit.Assert.assertThat; @@ -102,20 +100,14 @@ public class YARNITCase extends YarnTestBase { jobGraph.addJar(new org.apache.flink.core.fs.Path(testingJar.toURI())); - ApplicationId applicationId = null; - ClusterClient clusterClient = null; + try (ClusterClient clusterClient = yarnClusterDescriptor.deployJobCluster( + clusterSpecification, + jobGraph, + false)) { - try { - clusterClient = yarnClusterDescriptor.deployJobCluster( - clusterSpecification, - jobGraph, - false); - applicationId = clusterClient.getClusterId(); + ApplicationId applicationId = clusterClient.getClusterId(); - assertThat(clusterClient, is(instanceOf(RestClusterClient.class))); - final RestClusterClient restClusterClient = (RestClusterClient) clusterClient; - - final CompletableFuture jobResultCompletableFuture = restClusterClient.requestJobResult(jobGraph.getJobID()); + final CompletableFuture jobResultCompletableFuture = clusterClient.requestJobResult(jobGraph.getJobID()); final JobResult jobResult = jobResultCompletableFuture.get(); @@ -123,10 +115,6 @@ public class YARNITCase extends YarnTestBase { assertThat(jobResult.getSerializedThrowable().isPresent(), is(false)); waitApplicationFinishedElseKillIt(applicationId, yarnAppTerminateTimeout, yarnClusterDescriptor); - } finally { - if (clusterClient != null) { - clusterClient.close(); - } } } }); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/FakeClusterClient.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/FakeClusterClient.java index 63219520fb..82a9ea7d4a 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/FakeClusterClient.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/FakeClusterClient.java @@ -25,11 +25,13 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.util.OptionalFailure; import org.apache.hadoop.yarn.api.records.ApplicationId; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.Collection; @@ -61,14 +63,17 @@ public class FakeClusterClient extends ClusterClient { throw new UnsupportedOperationException("Not needed in test."); } + @Override public CompletableFuture getJobStatus(JobID jobId) { throw new UnsupportedOperationException("Not needed in test."); } + @Override public void cancel(JobID jobId) { // no op } + @Override public String cancelWithSavepoint(JobID jobId, @Nullable String savepointDirectory) { throw new UnsupportedOperationException("Not needed in test."); } @@ -78,19 +83,33 @@ public class FakeClusterClient extends ClusterClient { throw new UnsupportedOperationException("Not needed in test."); } + @Override public CompletableFuture triggerSavepoint(JobID jobId, @Nullable String savepointDirectory) { throw new UnsupportedOperationException("Not needed in test."); } + @Override public CompletableFuture disposeSavepoint(String savepointPath) { return CompletableFuture.completedFuture(Acknowledge.get()); } + @Override public CompletableFuture> listJobs() { return CompletableFuture.completedFuture(Collections.emptyList()); } + @Override public Map> getAccumulators(JobID jobID, ClassLoader loader) { return Collections.emptyMap(); } + + @Override + public CompletableFuture submitJob(@Nonnull JobGraph jobGraph) { + throw new UnsupportedOperationException("Not needed in test."); + } + + @Override + public CompletableFuture requestJobResult(@Nonnull JobID jobId) { + throw new UnsupportedOperationException("Not needed in test."); + } } -- Gitee From af37ae7607f93c3dd14f8eca55dd69af0dc3675d Mon Sep 17 00:00:00 2001 From: Wei Zhong Date: Fri, 20 Sep 2019 15:28:59 +0800 Subject: [PATCH 002/268] [FLINK-14140][python] Fix the broken flink logo in flink python shell. This closes #9725 --- flink-python/pyflink/shell.py | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/flink-python/pyflink/shell.py b/flink-python/pyflink/shell.py index 87eb21d967..8fabd496e5 100644 --- a/flink-python/pyflink/shell.py +++ b/flink-python/pyflink/shell.py @@ -40,16 +40,11 @@ print("Using Python version %s (%s, %s)" % ( welcome_msg = u''' \u2592\u2593\u2588\u2588\u2593\u2588\u2588\u2592 - \u2593\u2588\u2588\u2588\u2588\u2592\u2592\u2588\u2593 \ - \u2592\u2593\u2588\u2588\u2588\u2593\u2592 - \u2593\u2588\u2588\u2588\u2593\u2591\u2591 \u2592 \ - \u2592\u2592\u2593\u2588\u2588\u2592 \u2592 - \u2591\u2588\u2588\u2592 \u2592\u2592\u2593\u2593\u2588 \ - \u2593\u2593\u2592\u2591 \u2592\u2588\u2588\u2588\u2588 - \u2588\u2588\u2592 \u2591\u2592\u2593\u2588\u2588 \ - \u2588\u2592 \u2592\u2588\u2592\u2588\u2592 - \u2591\u2593\u2588 \u2588\u2588\u2588 \u2593 \ - \u2591\u2592\u2588\u2588 + \u2593\u2588\u2588\u2588\u2588\u2592\u2592\u2588\u2593\u2592\u2593\u2588\u2588\u2588\u2593\u2592 + \u2593\u2588\u2588\u2588\u2593\u2591\u2591 \u2592\u2592\u2592\u2593\u2588\u2588\u2592 \u2592 + \u2591\u2588\u2588\u2592 \u2592\u2592\u2593\u2593\u2588\u2593\u2593\u2592\u2591 \u2592\u2588\u2588\u2588\u2588 + \u2588\u2588\u2592 \u2591\u2592\u2593\u2588\u2588\u2588\u2592 \u2592\u2588\u2592\u2588\u2592 + \u2591\u2593\u2588 \u2588\u2588\u2588 \u2593\u2591\u2592\u2588\u2588 \u2593\u2588 \u2592\u2592\u2592\u2592\u2592\u2593\u2588\u2588\u2593\u2591\u2592\u2591\u2593\u2593\u2588 \u2588\u2591 \u2588 \u2592\u2592\u2591 \u2588\u2588\u2588\u2593\u2593\u2588 \u2592\u2588\u2592\u2592\u2592 \u2588\u2588\u2588\u2588\u2591 \u2592\u2593\u2588\u2593 \u2588\u2588\u2592\u2592\u2592 \u2593\u2588\u2588\u2588\u2592 -- Gitee From 8155d465520c4b616866d35395c3b10f7e809b78 Mon Sep 17 00:00:00 2001 From: Wei Zhong Date: Fri, 20 Sep 2019 19:13:47 +0800 Subject: [PATCH 003/268] [FLINK-14150][python] Clean up the __pycache__ directories and other empty directories in flink-python source code folder before packaging. This closes #9729. --- flink-python/pom.xml | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/flink-python/pom.xml b/flink-python/pom.xml index 4c63f39561..fcbdb0c4d6 100644 --- a/flink-python/pom.xml +++ b/flink-python/pom.xml @@ -197,9 +197,15 @@ under the License. - + + includes="**/*.pyc,**/__pycache__"/> + + + + + + @@ -217,9 +223,15 @@ under the License. - + + includes="**/*.pyc,**/__pycache__"/> + + + + + + -- Gitee From adfe011bc6fed36e30b3078bd3b6dbc0953f2ddf Mon Sep 17 00:00:00 2001 From: Ying Date: Fri, 30 Aug 2019 00:44:27 -0700 Subject: [PATCH 004/268] [FLINK-13864][fs-connector] Make StreamingFileSink extensible This PR makes the StreamingFileSink protected and the builders mutable so that they can be subclassed. In order for the user to subclass the StreamingFileSink, he has to override the forRowFormat/forBulkFormat depending on his needs and the corresponding builder so its the build() method returns the subclass and not the original StreamingFileSink. --- .../sink/filesystem/StreamingFileSink.java | 180 ++++++++++-------- .../sink/filesystem/BulkWriterTest.java | 80 +++++++- .../LocalStreamingFileSinkTest.java | 62 ++++++ .../functions/sink/filesystem/TestUtils.java | 139 ++++++++++++-- 4 files changed, 367 insertions(+), 94 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java index 75692dde1c..ed7e926424 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java @@ -110,7 +110,7 @@ public class StreamingFileSink private final long bucketCheckInterval; - private final StreamingFileSink.BucketsBuilder bucketsBuilder; + private final StreamingFileSink.BucketsBuilder> bucketsBuilder; // --------------------------- runtime fields ----------------------------- @@ -125,11 +125,24 @@ public class StreamingFileSink private transient ListState maxPartCountersState; /** - * Creates a new {@code StreamingFileSink} that writes files to the given base directory. + * Creates a new {@code StreamingFileSink} that writes files in row-based format to the given base directory. */ protected StreamingFileSink( - final StreamingFileSink.BucketsBuilder bucketsBuilder, - final long bucketCheckInterval) { + final RowFormatBuilder> bucketsBuilder, + final long bucketCheckInterval) { + + Preconditions.checkArgument(bucketCheckInterval > 0L); + + this.bucketsBuilder = Preconditions.checkNotNull(bucketsBuilder); + this.bucketCheckInterval = bucketCheckInterval; + } + + /** + * Creates a new {@code StreamingFileSink} that writes files in bulk-encoded format to the given base directory. + */ + protected StreamingFileSink( + final BulkFormatBuilder> bucketsBuilder, + final long bucketCheckInterval) { Preconditions.checkArgument(bucketCheckInterval > 0L); @@ -149,7 +162,7 @@ public class StreamingFileSink * @return The builder where the remaining of the configuration parameters for the sink can be configured. * In order to instantiate the sink, call {@link RowFormatBuilder#build()} after specifying the desired parameters. */ - public static StreamingFileSink.RowFormatBuilder forRowFormat( + public static StreamingFileSink.RowFormatBuilder> forRowFormat( final Path basePath, final Encoder encoder) { return new StreamingFileSink.RowFormatBuilder<>(basePath, encoder, new DateTimeBucketAssigner<>()); } @@ -162,7 +175,7 @@ public class StreamingFileSink * @return The builder where the remaining of the configuration parameters for the sink can be configured. * In order to instantiate the sink, call {@link RowFormatBuilder#build()} after specifying the desired parameters. */ - public static StreamingFileSink.BulkFormatBuilder forBulkFormat( + public static StreamingFileSink.BulkFormatBuilder> forBulkFormat( final Path basePath, final BulkWriter.Factory writerFactory) { return new StreamingFileSink.BulkFormatBuilder<>(basePath, writerFactory, new DateTimeBucketAssigner<>()); } @@ -170,10 +183,17 @@ public class StreamingFileSink /** * The base abstract class for the {@link RowFormatBuilder} and {@link BulkFormatBuilder}. */ - protected abstract static class BucketsBuilder implements Serializable { + private abstract static class BucketsBuilder> implements Serializable { private static final long serialVersionUID = 1L; + protected static final long DEFAULT_BUCKET_CHECK_INTERVAL = 60L * 1000L; + + @SuppressWarnings("unchecked") + protected T self() { + return (T) this; + } + abstract Buckets createBuckets(final int subtaskIndex) throws IOException; } @@ -181,31 +201,31 @@ public class StreamingFileSink * A builder for configuring the sink for row-wise encoding formats. */ @PublicEvolving - public static class RowFormatBuilder extends StreamingFileSink.BucketsBuilder { + public static class RowFormatBuilder> extends StreamingFileSink.BucketsBuilder { private static final long serialVersionUID = 1L; - private final long bucketCheckInterval; + private long bucketCheckInterval; private final Path basePath; - private final Encoder encoder; + private Encoder encoder; - private final BucketAssigner bucketAssigner; + private BucketAssigner bucketAssigner; - private final RollingPolicy rollingPolicy; + private RollingPolicy rollingPolicy; - private final BucketFactory bucketFactory; + private BucketFactory bucketFactory; - private final String partFilePrefix; + private String partFilePrefix; - private final String partFileSuffix; + private String partFileSuffix; - RowFormatBuilder(Path basePath, Encoder encoder, BucketAssigner bucketAssigner) { - this(basePath, encoder, bucketAssigner, DefaultRollingPolicy.builder().build(), 60L * 1000L, new DefaultBucketFactoryImpl<>(), PartFileConfig.DEFAULT_PART_PREFIX, PartFileConfig.DEFAULT_PART_SUFFIX); + protected RowFormatBuilder(Path basePath, Encoder encoder, BucketAssigner bucketAssigner) { + this(basePath, encoder, bucketAssigner, DefaultRollingPolicy.builder().build(), DEFAULT_BUCKET_CHECK_INTERVAL, new DefaultBucketFactoryImpl<>(), PartFileConfig.DEFAULT_PART_PREFIX, PartFileConfig.DEFAULT_PART_SUFFIX); } - private RowFormatBuilder( + protected RowFormatBuilder( Path basePath, Encoder encoder, BucketAssigner assigner, @@ -224,44 +244,38 @@ public class StreamingFileSink this.partFileSuffix = Preconditions.checkNotNull(partFileSuffix); } - /** - * Creates a new builder instance with the specified bucket check interval. The interval specifies how often - * time based {@link RollingPolicy}s will be checked/executed for the open buckets. - * @param interval Time interval in milliseconds - * @return A new builder with the check interval set. - */ - public StreamingFileSink.RowFormatBuilder withBucketCheckInterval(final long interval) { - return new RowFormatBuilder<>(basePath, encoder, bucketAssigner, rollingPolicy, interval, bucketFactory, partFilePrefix, partFileSuffix); + public long getBucketCheckInterval() { + return bucketCheckInterval; } - /** - * Creates a new builder instance with the specified {@link BucketAssigner}. - * @param assigner @{@link BucketAssigner} to be used. - * @return A new builder with the assigner set. - */ - public StreamingFileSink.RowFormatBuilder withBucketAssigner(final BucketAssigner assigner) { - return new RowFormatBuilder<>(basePath, encoder, Preconditions.checkNotNull(assigner), rollingPolicy, bucketCheckInterval, bucketFactory, partFilePrefix, partFileSuffix); + public T withBucketCheckInterval(final long interval) { + this.bucketCheckInterval = interval; + return self(); } - /** - * Creates a new builder instance with the specified {@link RollingPolicy} set for the bucketing logic. - * @param policy {@link RollingPolicy} to be applied - * @return A new builder with the check interval set. - */ - public StreamingFileSink.RowFormatBuilder withRollingPolicy(final RollingPolicy policy) { - return new RowFormatBuilder<>(basePath, encoder, bucketAssigner, Preconditions.checkNotNull(policy), bucketCheckInterval, bucketFactory, partFilePrefix, partFileSuffix); + public T withBucketAssigner(final BucketAssigner assigner) { + this.bucketAssigner = Preconditions.checkNotNull(assigner); + return self(); } - public StreamingFileSink.RowFormatBuilder withBucketAssignerAndPolicy(final BucketAssigner assigner, final RollingPolicy policy) { - return new RowFormatBuilder<>(basePath, encoder, Preconditions.checkNotNull(assigner), Preconditions.checkNotNull(policy), bucketCheckInterval, new DefaultBucketFactoryImpl<>(), partFilePrefix, partFileSuffix); + public T withRollingPolicy(final RollingPolicy policy) { + this.rollingPolicy = Preconditions.checkNotNull(policy); + return self(); } - public StreamingFileSink.RowFormatBuilder withPartFilePrefix(final String partPrefix) { - return new RowFormatBuilder<>(basePath, encoder, bucketAssigner, rollingPolicy, bucketCheckInterval, bucketFactory, partPrefix, partFileSuffix); + public T withPartFilePrefix(final String partPrefix) { + this.partFilePrefix = partPrefix; + return self(); } - public StreamingFileSink.RowFormatBuilder withPartFileSuffix(final String partSuffix) { - return new RowFormatBuilder<>(basePath, encoder, bucketAssigner, rollingPolicy, bucketCheckInterval, bucketFactory, partFilePrefix, partSuffix); + public T withPartFileSuffix(final String partSuffix) { + this.partFileSuffix = partSuffix; + return self(); + } + + public StreamingFileSink.RowFormatBuilder> withNewBucketAssignerAndPolicy(final BucketAssigner assigner, final RollingPolicy policy) { + Preconditions.checkState(bucketFactory.getClass() == DefaultBucketFactoryImpl.class, "newBuilderWithBucketAssignerAndPolicy() cannot be called after specifying a customized bucket factory"); + return new RowFormatBuilder<>(basePath, encoder, Preconditions.checkNotNull(assigner), Preconditions.checkNotNull(policy), bucketCheckInterval, new DefaultBucketFactoryImpl<>(), partFilePrefix, partFileSuffix); } /** Creates the actual sink. */ @@ -269,6 +283,12 @@ public class StreamingFileSink return new StreamingFileSink<>(this, bucketCheckInterval); } + @VisibleForTesting + T withBucketFactory(final BucketFactory factory) { + this.bucketFactory = Preconditions.checkNotNull(factory); + return self(); + } + @Override Buckets createBuckets(int subtaskIndex) throws IOException { return new Buckets<>( @@ -280,40 +300,35 @@ public class StreamingFileSink subtaskIndex, new PartFileConfig(partFilePrefix, partFileSuffix)); } - - @VisibleForTesting - StreamingFileSink.RowFormatBuilder withBucketFactory(final BucketFactory factory) { - return new RowFormatBuilder<>(basePath, encoder, bucketAssigner, rollingPolicy, bucketCheckInterval, Preconditions.checkNotNull(factory), partFilePrefix, partFileSuffix); - } } /** * A builder for configuring the sink for bulk-encoding formats, e.g. Parquet/ORC. */ @PublicEvolving - public static class BulkFormatBuilder extends StreamingFileSink.BucketsBuilder { + public static class BulkFormatBuilder> extends StreamingFileSink.BucketsBuilder { private static final long serialVersionUID = 1L; - private final long bucketCheckInterval; + private long bucketCheckInterval; private final Path basePath; - private final BulkWriter.Factory writerFactory; + private BulkWriter.Factory writerFactory; - private final BucketAssigner bucketAssigner; + private BucketAssigner bucketAssigner; - private final BucketFactory bucketFactory; + private BucketFactory bucketFactory; - private final String partFilePrefix; + private String partFilePrefix; - private final String partFileSuffix; + private String partFileSuffix; - BulkFormatBuilder(Path basePath, BulkWriter.Factory writerFactory, BucketAssigner assigner) { - this(basePath, writerFactory, assigner, 60L * 1000L, new DefaultBucketFactoryImpl<>(), PartFileConfig.DEFAULT_PART_PREFIX, PartFileConfig.DEFAULT_PART_SUFFIX); + protected BulkFormatBuilder(Path basePath, BulkWriter.Factory writerFactory, BucketAssigner assigner) { + this(basePath, writerFactory, assigner, DEFAULT_BUCKET_CHECK_INTERVAL, new DefaultBucketFactoryImpl<>(), PartFileConfig.DEFAULT_PART_PREFIX, PartFileConfig.DEFAULT_PART_SUFFIX); } - private BulkFormatBuilder( + protected BulkFormatBuilder( Path basePath, BulkWriter.Factory writerFactory, BucketAssigner assigner, @@ -330,34 +345,39 @@ public class StreamingFileSink this.partFileSuffix = Preconditions.checkNotNull(partFileSuffix); } - /** - * Currently bulk formats always use the {@link OnCheckpointRollingPolicy} therefore this settings does - * not have any effect. - */ - public StreamingFileSink.BulkFormatBuilder withBucketCheckInterval(long interval) { - return new BulkFormatBuilder<>(basePath, writerFactory, bucketAssigner, interval, bucketFactory, partFilePrefix, partFileSuffix); + public long getBucketCheckInterval() { + return bucketCheckInterval; } - /** - * Creates a new builder instance with the specified {@link BucketAssigner}. - * @param assigner @{@link BucketAssigner} to be used. - * @return A new builder with the assigner set. - */ - public StreamingFileSink.BulkFormatBuilder withBucketAssigner(BucketAssigner assigner) { - return new BulkFormatBuilder<>(basePath, writerFactory, Preconditions.checkNotNull(assigner), bucketCheckInterval, new DefaultBucketFactoryImpl<>(), partFilePrefix, partFileSuffix); + public T withBucketCheckInterval(long interval) { + this.bucketCheckInterval = interval; + return self(); + } + + public T withBucketAssigner(BucketAssigner assigner) { + this.bucketAssigner = Preconditions.checkNotNull(assigner); + return self(); } @VisibleForTesting - StreamingFileSink.BulkFormatBuilder withBucketFactory(final BucketFactory factory) { - return new BulkFormatBuilder<>(basePath, writerFactory, bucketAssigner, bucketCheckInterval, Preconditions.checkNotNull(factory), partFilePrefix, partFileSuffix); + T withBucketFactory(final BucketFactory factory) { + this.bucketFactory = Preconditions.checkNotNull(factory); + return self(); } - public StreamingFileSink.BulkFormatBuilder withPartFilePrefix(final String partPrefix) { - return new BulkFormatBuilder<>(basePath, writerFactory, bucketAssigner, bucketCheckInterval, bucketFactory, partPrefix, partFileSuffix); + public T withPartFilePrefix(final String partPrefix) { + this.partFilePrefix = partPrefix; + return self(); + } + + public T withPartFileSuffix(final String partSuffix) { + this.partFileSuffix = partSuffix; + return self(); } - public StreamingFileSink.BulkFormatBuilder withPartFileSuffix(final String partSuffix) { - return new BulkFormatBuilder<>(basePath, writerFactory, bucketAssigner, bucketCheckInterval, bucketFactory, partFilePrefix, partSuffix); + public StreamingFileSink.BulkFormatBuilder> withNewBucketAssigner(final BucketAssigner assigner) { + Preconditions.checkState(bucketFactory.getClass() == DefaultBucketFactoryImpl.class, "newBuilderWithBucketAssigner() cannot be called after specifying a customized bucket factory"); + return new BulkFormatBuilder<>(basePath, writerFactory, Preconditions.checkNotNull(assigner), bucketCheckInterval, new DefaultBucketFactoryImpl<>(), partFilePrefix, partFileSuffix); } /** Creates the actual sink. */ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkWriterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkWriterTest.java index 3c1b9badc8..bab18aac3b 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkWriterTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkWriterTest.java @@ -61,7 +61,28 @@ public class BulkWriterTest extends TestLogger { new TestBulkWriterFactory(), new DefaultBucketFactoryImpl<>()) ) { - testPartFiles(testHarness, outDir, ".part-0-0.inprogress", ".part-0-1.inprogress"); + testPartFilesWithStringBucketer(testHarness, outDir, ".part-0-0.inprogress", ".part-0-1.inprogress"); + } + } + + @Test + public void testCustomBulkWriterWithBucketAssigner() throws Exception { + final File outDir = TEMP_FOLDER.newFolder(); + + // we set the max bucket size to small so that we can know when it rolls + try ( + OneInputStreamOperatorTestHarness, Object> testHarness = + TestUtils.createTestSinkWithCustomizedBulkEncoder( + outDir, + 1, + 0, + 10L, + // use a customized bucketer with Integer bucket ID + new TestUtils.TupleToIntegerBucketer(), + new TestBulkWriterFactory(), + new DefaultBucketFactoryImpl<>()) + ) { + testPartFilesWithIntegerBucketer(testHarness, outDir, ".part-0-0.inprogress", ".part-0-1.inprogress", ".part-0-2.inprogress"); } } @@ -83,11 +104,11 @@ public class BulkWriterTest extends TestLogger { "prefix", ".ext") ) { - testPartFiles(testHarness, outDir, ".prefix-0-0.ext.inprogress", ".prefix-0-1.ext.inprogress"); + testPartFilesWithStringBucketer(testHarness, outDir, ".prefix-0-0.ext.inprogress", ".prefix-0-1.ext.inprogress"); } } - private void testPartFiles( + private void testPartFilesWithStringBucketer( OneInputStreamOperatorTestHarness, Object> testHarness, File outDir, String partFileName1, @@ -122,6 +143,8 @@ public class BulkWriterTest extends TestLogger { fileCounter++; Assert.assertEquals("test1@2\ntest1@3\n", fileContents.getValue()); } + // check bucket name + Assert.assertEquals("test1", fileContents.getKey().getParentFile().getName()); } Assert.assertEquals(2L, fileCounter); @@ -131,6 +154,57 @@ public class BulkWriterTest extends TestLogger { TestUtils.checkLocalFs(outDir, 0, 2); } + private void testPartFilesWithIntegerBucketer( + OneInputStreamOperatorTestHarness, Object> testHarness, + File outDir, + String partFileName1, + String partFileName2, + String partFileName3) throws Exception { + + testHarness.setup(); + testHarness.open(); + + // this creates a new bucket "test1" and part-0-0 + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 1), 1L)); + TestUtils.checkLocalFs(outDir, 1, 0); + + // we take a checkpoint so we roll. + testHarness.snapshot(1L, 1L); + + // these will close part-0-0 and open part-0-1 and part-0-2 + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 2), 2L)); + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 3), 3L)); + + // we take a checkpoint so we roll again. + testHarness.snapshot(2L, 2L); + + TestUtils.checkLocalFs(outDir, 3, 0); + + Map contents = TestUtils.getFileContentByPath(outDir); + int fileCounter = 0; + for (Map.Entry fileContents : contents.entrySet()) { + if (fileContents.getKey().getName().contains(partFileName1)) { + fileCounter++; + Assert.assertEquals("test1@1\n", fileContents.getValue()); + Assert.assertEquals("1", fileContents.getKey().getParentFile().getName()); + } else if (fileContents.getKey().getName().contains(partFileName2)) { + fileCounter++; + Assert.assertEquals("test1@2\n", fileContents.getValue()); + Assert.assertEquals("2", fileContents.getKey().getParentFile().getName()); + } else if (fileContents.getKey().getName().contains(partFileName3)) { + fileCounter++; + Assert.assertEquals("test1@3\n", fileContents.getValue()); + Assert.assertEquals("3", fileContents.getKey().getParentFile().getName()); + } + } + Assert.assertEquals(3L, fileCounter); + + // we acknowledge the latest checkpoint, so everything should be published. + testHarness.notifyOfCompletedCheckpoint(2L); + + TestUtils.checkLocalFs(outDir, 0, 3); + } + /** * A {@link BulkWriter} used for the tests. */ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/LocalStreamingFileSinkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/LocalStreamingFileSinkTest.java index 5b5aff35db..8203375679 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/LocalStreamingFileSinkTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/LocalStreamingFileSinkTest.java @@ -20,6 +20,9 @@ package org.apache.flink.streaming.api.functions.sink.filesystem; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.functions.sink.filesystem.TestUtils.Tuple2Encoder; +import org.apache.flink.streaming.api.functions.sink.filesystem.TestUtils.TupleToIntegerBucketer; +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; @@ -403,6 +406,65 @@ public class LocalStreamingFileSinkTest extends TestLogger { TestUtils.checkLocalFs(outDir, 1, 3); } + @Test + public void testClosingWithCustomizedBucketer() throws Exception { + final File outDir = TEMP_FOLDER.newFolder(); + final long partMaxSize = 2L; + final long inactivityInterval = 100L; + final RollingPolicy, Integer> rollingPolicy = + DefaultRollingPolicy + .builder() + .withMaxPartSize(partMaxSize) + .withRolloverInterval(inactivityInterval) + .withInactivityInterval(inactivityInterval) + .build(); + + try ( + OneInputStreamOperatorTestHarness, Object> testHarness = + TestUtils.createCustomizedRescalingTestSink(outDir, 1, 0, 100L, new TupleToIntegerBucketer(), new Tuple2Encoder(), rollingPolicy, new DefaultBucketFactoryImpl<>()); + ) { + testHarness.setup(); + testHarness.open(); + + testHarness.setProcessingTime(0L); + + testHarness.processElement(new StreamRecord<>(Tuple2.of("test1", 1), 1L)); + testHarness.processElement(new StreamRecord<>(Tuple2.of("test2", 2), 1L)); + TestUtils.checkLocalFs(outDir, 2, 0); + + // this is to check the inactivity threshold + testHarness.setProcessingTime(101L); + TestUtils.checkLocalFs(outDir, 2, 0); + + testHarness.processElement(new StreamRecord<>(Tuple2.of("test3", 3), 1L)); + TestUtils.checkLocalFs(outDir, 3, 0); + + testHarness.snapshot(0L, 1L); + TestUtils.checkLocalFs(outDir, 3, 0); + + testHarness.notifyOfCompletedCheckpoint(0L); + TestUtils.checkLocalFs(outDir, 0, 3); + + testHarness.processElement(new StreamRecord<>(Tuple2.of("test4", 4), 10L)); + TestUtils.checkLocalFs(outDir, 1, 3); + + testHarness.snapshot(1L, 0L); + testHarness.notifyOfCompletedCheckpoint(1L); + } + + // at close all files moved to final. + TestUtils.checkLocalFs(outDir, 0, 4); + + // check file content and bucket ID. + Map contents = TestUtils.getFileContentByPath(outDir); + for (Map.Entry fileContents : contents.entrySet()) { + Integer bucketId = Integer.parseInt(fileContents.getKey().getParentFile().getName()); + + Assert.assertTrue(bucketId >= 1 && bucketId <= 4); + Assert.assertEquals(String.format("test%d@%d\n", bucketId, bucketId), fileContents.getValue()); + } + } + @Test public void testScalingDownAndMergingOfStates() throws Exception { final File outDir = TEMP_FOLDER.newFolder(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/TestUtils.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/TestUtils.java index 55ba6d33e1..c17a2d6a24 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/TestUtils.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/TestUtils.java @@ -38,6 +38,9 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collection; @@ -68,25 +71,18 @@ public class TestUtils { .withInactivityInterval(inactivityInterval) .build(); - final BucketAssigner, String> bucketer = new TupleToStringBucketer(); - - final Encoder> encoder = (element, stream) -> { - stream.write((element.f0 + '@' + element.f1).getBytes(StandardCharsets.UTF_8)); - stream.write('\n'); - }; - - return createCustomRescalingTestSink( + return createRescalingTestSink( outDir, totalParallelism, taskIdx, 10L, - bucketer, - encoder, + new TupleToStringBucketer(), + new Tuple2Encoder(), rollingPolicy, new DefaultBucketFactoryImpl<>()); } - static OneInputStreamOperatorTestHarness, Object> createCustomRescalingTestSink( + static OneInputStreamOperatorTestHarness, Object> createRescalingTestSink( final File outDir, final int totalParallelism, final int taskIdx, @@ -107,6 +103,26 @@ public class TestUtils { return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), MAX_PARALLELISM, totalParallelism, taskIdx); } + static OneInputStreamOperatorTestHarness, Object> createCustomizedRescalingTestSink( + final File outDir, + final int totalParallelism, + final int taskIdx, + final long bucketCheckInterval, + final BucketAssigner, ID> bucketer, + final Encoder> writer, + final RollingPolicy, ID> rollingPolicy, + final BucketFactory, ID> bucketFactory) throws Exception { + + StreamingFileSink> sink = StreamingFileSink + .forRowFormat(new Path(outDir.toURI()), writer) + .withNewBucketAssignerAndPolicy(bucketer, rollingPolicy) + .withBucketCheckInterval(bucketCheckInterval) + .withBucketFactory(bucketFactory) + .build(); + + return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), MAX_PARALLELISM, totalParallelism, taskIdx); + } + static OneInputStreamOperatorTestHarness, Object> createTestSinkWithBulkEncoder( final File outDir, final int totalParallelism, @@ -151,6 +167,50 @@ public class TestUtils { return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), MAX_PARALLELISM, totalParallelism, taskIdx); } + static OneInputStreamOperatorTestHarness, Object> createTestSinkWithCustomizedBulkEncoder( + final File outDir, + final int totalParallelism, + final int taskIdx, + final long bucketCheckInterval, + final BucketAssigner, ID> bucketer, + final BulkWriter.Factory> writer, + final BucketFactory, ID> bucketFactory) throws Exception { + + return createTestSinkWithCustomizedBulkEncoder( + outDir, + totalParallelism, + taskIdx, + bucketCheckInterval, + bucketer, + writer, + bucketFactory, + PartFileConfig.DEFAULT_PART_PREFIX, + PartFileConfig.DEFAULT_PART_SUFFIX); + } + + static OneInputStreamOperatorTestHarness, Object> createTestSinkWithCustomizedBulkEncoder( + final File outDir, + final int totalParallelism, + final int taskIdx, + final long bucketCheckInterval, + final BucketAssigner, ID> bucketer, + final BulkWriter.Factory> writer, + final BucketFactory, ID> bucketFactory, + final String partFilePrefix, + final String partFileSuffix) throws Exception { + + StreamingFileSink> sink = StreamingFileSink + .forBulkFormat(new Path(outDir.toURI()), writer) + .withNewBucketAssigner(bucketer) + .withBucketCheckInterval(bucketCheckInterval) + .withBucketFactory(bucketFactory) + .withPartFilePrefix(partFilePrefix) + .withPartFileSuffix(partFileSuffix) + .build(); + + return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), MAX_PARALLELISM, totalParallelism, taskIdx); + } + static void checkLocalFs(File outDir, int expectedInProgress, int expectedCompleted) { int inProgress = 0; int finished = 0; @@ -181,6 +241,21 @@ public class TestUtils { return contents; } + /** + * A simple {@link Encoder} that encodes {@code Tuple2} object. + */ + static class Tuple2Encoder implements Encoder> { + @Override + public void encode(Tuple2 element, OutputStream stream) throws IOException { + stream.write((element.f0 + '@' + element.f1).getBytes(StandardCharsets.UTF_8)); + stream.write('\n'); + } + } + + /** + * A simple {@link BucketAssigner} that returns the first (String) element of a {@code Tuple2} + * object as the bucket id. + */ static class TupleToStringBucketer implements BucketAssigner, String> { private static final long serialVersionUID = 1L; @@ -196,6 +271,48 @@ public class TestUtils { } } + /** + * A simple {@link BucketAssigner} that returns the second (Integer) element of a {@code Tuple2} + * object as the bucket id. + */ + static class TupleToIntegerBucketer implements BucketAssigner, Integer> { + + private static final long serialVersionUID = 1L; + + @Override + public Integer getBucketId(Tuple2 element, Context context) { + return element.f1; + } + + @Override + public SimpleVersionedSerializer getSerializer() { + return new SimpleVersionedIntegerSerializer(); + } + } + + private static final class SimpleVersionedIntegerSerializer implements SimpleVersionedSerializer { + static final int VERSION = 1; + + private SimpleVersionedIntegerSerializer() { + } + + public int getVersion() { + return 1; + } + + public byte[] serialize(Integer value) { + byte[] bytes = new byte[4]; + ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).putInt(value); + return bytes; + } + + public Integer deserialize(int version, byte[] serialized) throws IOException { + Assert.assertEquals(1L, (long) version); + Assert.assertEquals(4L, serialized.length); + return ByteBuffer.wrap(serialized).order(ByteOrder.LITTLE_ENDIAN).getInt(); + } + } + /** * A simple {@link BucketAssigner} that accepts {@code String}'s * and returns the element itself as the bucket id. -- Gitee From c83c18671bc0056a341877f312ba293ae5811953 Mon Sep 17 00:00:00 2001 From: David Anderson Date: Sat, 21 Sep 2019 10:51:02 +0200 Subject: [PATCH 005/268] [FLINK-14160][docs] Describe --backpressure option for Operations Playground. This closes #9739. --- .../flink-operations-playground.md | 17 ++++++++++++++--- .../flink-operations-playground.zh.md | 17 ++++++++++++++--- 2 files changed, 28 insertions(+), 6 deletions(-) diff --git a/docs/getting-started/docker-playgrounds/flink-operations-playground.md b/docs/getting-started/docker-playgrounds/flink-operations-playground.md index 38a0848237..c9f76757aa 100644 --- a/docs/getting-started/docker-playgrounds/flink-operations-playground.md +++ b/docs/getting-started/docker-playgrounds/flink-operations-playground.md @@ -136,7 +136,7 @@ will show you how to interact with the Flink Cluster and demonstrate some of Fli ### Flink WebUI -The most natural starting point to observe your Flink Cluster is the Web UI exposed under +The most natural starting point to observe your Flink Cluster is the WebUI exposed under [http://localhost:8081](http://localhost:8081). If everything went well, you'll see that the cluster initially consists of one TaskManager and executes a Job called *Click Event Count*. @@ -802,8 +802,8 @@ TaskManager metrics); ## Variants -You might have noticed that the *Click Event Count* was always started with `--checkpointing` and -`--event-time` program arguments. By omitting these in the command of the *client* container in the +You might have noticed that the *Click Event Count* application was always started with `--checkpointing` +and `--event-time` program arguments. By omitting these in the command of the *client* container in the `docker-compose.yaml`, you can change the behavior of the Job. * `--checkpointing` enables [checkpoint]({{ site.baseurl }}/internals/stream_checkpointing.html), @@ -815,3 +815,14 @@ lost. Job. When disabled, the Job will assign events to windows based on the wall-clock time instead of the timestamp of the `ClickEvent`. Consequently, the number of events per window will not be exactly one thousand anymore. + +The *Click Event Count* application also has another option, turned off by default, that you can +enable to explore the behavior of this job under backpressure. You can add this option in the +command of the *client* container in `docker-compose.yaml`. + +* `--backpressure` adds an additional operator into the middle of the job that causes severe backpressure +during even-numbered minutes (e.g., during 10:12, but not during 10:13). This can be observed by +inspecting various [network metrics]({{ site.baseurl }}/monitoring/metrics.html#default-shuffle-service) +such as `outputQueueLength` and `outPoolUsage`, and/or by using the +[backpressure monitoring]({{ site.baseurl }}/monitoring/back_pressure.html#monitoring-back-pressure) +available in the WebUI. \ No newline at end of file diff --git a/docs/getting-started/docker-playgrounds/flink-operations-playground.zh.md b/docs/getting-started/docker-playgrounds/flink-operations-playground.zh.md index 38a0848237..c9f76757aa 100644 --- a/docs/getting-started/docker-playgrounds/flink-operations-playground.zh.md +++ b/docs/getting-started/docker-playgrounds/flink-operations-playground.zh.md @@ -136,7 +136,7 @@ will show you how to interact with the Flink Cluster and demonstrate some of Fli ### Flink WebUI -The most natural starting point to observe your Flink Cluster is the Web UI exposed under +The most natural starting point to observe your Flink Cluster is the WebUI exposed under [http://localhost:8081](http://localhost:8081). If everything went well, you'll see that the cluster initially consists of one TaskManager and executes a Job called *Click Event Count*. @@ -802,8 +802,8 @@ TaskManager metrics); ## Variants -You might have noticed that the *Click Event Count* was always started with `--checkpointing` and -`--event-time` program arguments. By omitting these in the command of the *client* container in the +You might have noticed that the *Click Event Count* application was always started with `--checkpointing` +and `--event-time` program arguments. By omitting these in the command of the *client* container in the `docker-compose.yaml`, you can change the behavior of the Job. * `--checkpointing` enables [checkpoint]({{ site.baseurl }}/internals/stream_checkpointing.html), @@ -815,3 +815,14 @@ lost. Job. When disabled, the Job will assign events to windows based on the wall-clock time instead of the timestamp of the `ClickEvent`. Consequently, the number of events per window will not be exactly one thousand anymore. + +The *Click Event Count* application also has another option, turned off by default, that you can +enable to explore the behavior of this job under backpressure. You can add this option in the +command of the *client* container in `docker-compose.yaml`. + +* `--backpressure` adds an additional operator into the middle of the job that causes severe backpressure +during even-numbered minutes (e.g., during 10:12, but not during 10:13). This can be observed by +inspecting various [network metrics]({{ site.baseurl }}/monitoring/metrics.html#default-shuffle-service) +such as `outputQueueLength` and `outPoolUsage`, and/or by using the +[backpressure monitoring]({{ site.baseurl }}/monitoring/back_pressure.html#monitoring-back-pressure) +available in the WebUI. \ No newline at end of file -- Gitee From f5b6952daf63eead293b856ba8e7955843d2af8f Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Sun, 18 Aug 2019 00:22:39 +0200 Subject: [PATCH 006/268] [hotfix][task] Refactor the process of checking input index for StreamOneInputProcessor#processInput We can check the input index inside StreamTaskNetworkInput while updating, so we could remove this logic in StreamOneInputProcessor#processInput for the preparation of processing input based on emitNext way. --- .../flink/streaming/runtime/io/StreamOneInputProcessor.java | 5 +---- .../flink/streaming/runtime/io/StreamTaskNetworkInput.java | 1 + 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java index 28aea1910f..a699c5aa55 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java @@ -143,10 +143,7 @@ public final class StreamOneInputProcessor implements StreamInputProcessor { public boolean processInput() throws Exception { StreamElement recordOrMark = input.pollNextNullable(); if (recordOrMark != null) { - int channel = input.getLastChannel(); - checkState(channel != StreamTaskInput.UNSPECIFIED); - - processElement(recordOrMark, channel); + processElement(recordOrMark, input.getLastChannel()); } checkFinished(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java index 0bd6e624aa..e739fd3b77 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java @@ -134,6 +134,7 @@ public final class StreamTaskNetworkInput implements StreamTaskInput { private void processBufferOrEvent(BufferOrEvent bufferOrEvent) throws IOException { if (bufferOrEvent.isBuffer()) { lastChannel = bufferOrEvent.getChannelIndex(); + checkState(lastChannel != StreamTaskInput.UNSPECIFIED); currentRecordDeserializer = recordDeserializers[lastChannel]; checkState(currentRecordDeserializer != null, "currentRecordDeserializer has already been released"); -- Gitee From 7aa06710c4a75b01e3837c16cf1013b427a3e134 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Tue, 27 Aug 2019 14:56:55 +0800 Subject: [PATCH 007/268] [hotfix][task] Remove unncessary SuppressWarnings from StreamOneInputProcessor --- .../flink/streaming/runtime/io/StreamOneInputProcessor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java index a699c5aa55..410e2be211 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java @@ -85,7 +85,6 @@ public final class StreamOneInputProcessor implements StreamInputProcessor { private final WatermarkGauge watermarkGauge; private final Counter numRecordsIn; - @SuppressWarnings("unchecked") public StreamOneInputProcessor( InputGate[] inputGates, TypeSerializer inputSerializer, -- Gitee From 0983d1f1aff11ae810d5a782d758160fc3d0448e Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Tue, 27 Aug 2019 15:14:07 +0800 Subject: [PATCH 008/268] [hotfix][task] Remove unused argument from constructor of StreamTaskNetworkInput --- .../flink/streaming/runtime/io/StreamTaskNetworkInput.java | 1 - .../flink/streaming/runtime/io/StreamTaskNetworkInputTest.java | 1 - 2 files changed, 2 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java index e739fd3b77..ff592629f8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java @@ -86,7 +86,6 @@ public final class StreamTaskNetworkInput implements StreamTaskInput { StreamTaskNetworkInput( CheckpointedInputGate checkpointedInputGate, TypeSerializer inputSerializer, - IOManager ioManager, int inputIndex, RecordDeserializer>[] recordDeserializers) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java index c5301b2f56..cdc70c5962 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java @@ -107,7 +107,6 @@ public class StreamTaskNetworkInputTest { new EmptyBufferStorage(), new CheckpointBarrierTracker(1)), inSerializer, - ioManager, 0, deserializers); -- Gitee From c727190878030f36fcd96743a83489f1b7e218dd Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Tue, 10 Sep 2019 13:42:36 +0800 Subject: [PATCH 009/268] [hotfix][network] Refactor the class name AsyncDataInput to PullingAsyncDataInput --- .../org/apache/flink/runtime/io/AvailabilityListener.java | 2 +- .../org/apache/flink/runtime/io/NullableAsyncDataInput.java | 6 +++--- .../io/{AsyncDataInput.java => PullingAsyncDataInput.java} | 2 +- .../runtime/io/network/partition/consumer/InputGate.java | 4 ++-- .../io/network/partition/consumer/InputGateTestBase.java | 6 +++--- .../flink/streaming/runtime/io/CheckpointedInputGate.java | 4 ++-- 6 files changed, 12 insertions(+), 12 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/io/{AsyncDataInput.java => PullingAsyncDataInput.java} (96%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityListener.java index f26a9194ad..f7e61d2d8d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityListener.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityListener.java @@ -23,7 +23,7 @@ import java.util.concurrent.CompletableFuture; /** * Interface defining couple of essential methods for listening on data availability using - * {@link CompletableFuture}. For usage check out for example {@link AsyncDataInput}. + * {@link CompletableFuture}. For usage check out for example {@link PullingAsyncDataInput}. */ @Internal public interface AvailabilityListener { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/NullableAsyncDataInput.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/NullableAsyncDataInput.java index f73970d05e..3c65e4413c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/NullableAsyncDataInput.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/NullableAsyncDataInput.java @@ -22,9 +22,9 @@ import org.apache.flink.annotation.Internal; import javax.annotation.Nullable; /** - * The variant of {@link AsyncDataInput} that for performance reasons returns {@code null} from - * {@link #pollNextNullable()} instead returning {@code Optional.empty()} from - * {@link AsyncDataInput#pollNext()}. + * The variant of {@link PullingAsyncDataInput} that for performance reasons returns {@code null} + * from {@link #pollNextNullable()} instead returning {@code Optional.empty()} from + * {@link PullingAsyncDataInput#pollNext()}. */ @Internal public interface NullableAsyncDataInput extends AvailabilityListener { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/AsyncDataInput.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/PullingAsyncDataInput.java similarity index 96% rename from flink-runtime/src/main/java/org/apache/flink/runtime/io/AsyncDataInput.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/io/PullingAsyncDataInput.java index 7f76e84370..f792c280a4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/AsyncDataInput.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/PullingAsyncDataInput.java @@ -50,7 +50,7 @@ import java.util.concurrent.CompletableFuture; * */ @Internal -public interface AsyncDataInput extends AvailabilityListener { +public interface PullingAsyncDataInput extends AvailabilityListener { /** * Poll the next element. This method should be non blocking. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java index 7d6ea818e8..6a9ac1744f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.io.network.partition.consumer; import org.apache.flink.runtime.event.TaskEvent; -import org.apache.flink.runtime.io.AsyncDataInput; +import org.apache.flink.runtime.io.PullingAsyncDataInput; import java.io.IOException; import java.util.Optional; @@ -69,7 +69,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; * will have an input gate attached to it. This will provide its input, which will consist of one * subpartition from each partition of the intermediate result. */ -public abstract class InputGate implements AsyncDataInput, AutoCloseable { +public abstract class InputGate implements PullingAsyncDataInput, AutoCloseable { protected CompletableFuture isAvailable = new CompletableFuture<>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java index b32ba89fd2..4069ec5faf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import org.apache.flink.runtime.io.AsyncDataInput; +import org.apache.flink.runtime.io.PullingAsyncDataInput; import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; @@ -69,7 +69,7 @@ public abstract class InputGateTestBase { assertTrue(isAvailable.isDone()); assertTrue(inputGateToTest.isAvailable().isDone()); - assertEquals(AsyncDataInput.AVAILABLE, inputGateToTest.isAvailable()); + assertEquals(PullingAsyncDataInput.AVAILABLE, inputGateToTest.isAvailable()); } protected void testIsAvailableAfterFinished( @@ -86,7 +86,7 @@ public abstract class InputGateTestBase { assertTrue(available.isDone()); assertTrue(inputGateToTest.isAvailable().isDone()); - assertEquals(AsyncDataInput.AVAILABLE, inputGateToTest.isAvailable()); + assertEquals(PullingAsyncDataInput.AVAILABLE, inputGateToTest.isAvailable()); } protected SingleInputGate createInputGate() { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java index 8e82a005a5..63dd16b267 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointedInputGate.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.runtime.io; import org.apache.flink.annotation.Internal; -import org.apache.flink.runtime.io.AsyncDataInput; +import org.apache.flink.runtime.io.PullingAsyncDataInput; import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; @@ -42,7 +42,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; * {@link CheckpointBarrier} from the {@link InputGate}. */ @Internal -public class CheckpointedInputGate implements AsyncDataInput { +public class CheckpointedInputGate implements PullingAsyncDataInput { private static final Logger LOG = LoggerFactory.getLogger(CheckpointedInputGate.class); -- Gitee From 4d42eaba43c39b301791920c76e8bdb3ca26970c Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Tue, 10 Sep 2019 12:17:44 +0800 Subject: [PATCH 010/268] [hotfix][task] Refactor the constrcutor of StreamTwoInputProcessor Migrate the construction of local temporary fields into TwoInputStreamTask and initialize the non-final fields during declaration in StreamTwoInputProcessor instead of inside construcotor. To do so we could make the constructor of StreamTwoInputProcessor short and fresh. --- .../runtime/io/StreamTwoInputProcessor.java | 75 ++++++------------- .../runtime/tasks/TwoInputStreamTask.java | 28 +++++-- 2 files changed, 44 insertions(+), 59 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index df5d186a09..4541fcbb62 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -20,11 +20,8 @@ package org.apache.flink.streaming.runtime.io; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; @@ -34,16 +31,13 @@ import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve; import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import org.apache.flink.streaming.runtime.tasks.OperatorChain; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.tasks.TwoInputStreamTask; import org.apache.flink.util.ExceptionUtils; import java.io.IOException; -import java.util.Collection; import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; /** * Input reader for {@link TwoInputStreamTask}. @@ -75,71 +69,46 @@ public final class StreamTwoInputProcessor implements StreamInputProce * Stream status for the two inputs. We need to keep track for determining when * to forward stream status changes downstream. */ - private StreamStatus firstStatus; - private StreamStatus secondStatus; + private StreamStatus firstStatus = StreamStatus.ACTIVE; + private StreamStatus secondStatus = StreamStatus.ACTIVE; - private int lastReadInputIndex; + /** Always try to read from the first input. */ + private int lastReadInputIndex = 1; private final Counter numRecordsIn; private boolean isPrepared; public StreamTwoInputProcessor( - Collection inputGates1, - Collection inputGates2, - TypeSerializer inputSerializer1, - TypeSerializer inputSerializer2, - StreamTask streamTask, - CheckpointingMode checkpointingMode, - Object lock, - IOManager ioManager, - Configuration taskManagerConfig, - StreamStatusMaintainer streamStatusMaintainer, - TwoInputStreamOperator streamOperator, - TwoInputSelectionHandler inputSelectionHandler, - WatermarkGauge input1WatermarkGauge, - WatermarkGauge input2WatermarkGauge, - String taskName, - OperatorChain operatorChain, - Counter numRecordsIn) throws IOException { - - this.streamOperator = checkNotNull(streamOperator); - this.inputSelectionHandler = checkNotNull(inputSelectionHandler); - - this.lock = checkNotNull(lock); + CheckpointedInputGate[] checkpointedInputGates, + TypeSerializer inputSerializer1, + TypeSerializer inputSerializer2, + Object lock, + IOManager ioManager, + StreamStatusMaintainer streamStatusMaintainer, + TwoInputStreamOperator streamOperator, + TwoInputSelectionHandler inputSelectionHandler, + WatermarkGauge input1WatermarkGauge, + WatermarkGauge input2WatermarkGauge, + OperatorChain operatorChain, + Counter numRecordsIn) { - InputGate unionedInputGate1 = InputGateUtil.createInputGate(inputGates1.toArray(new InputGate[0])); - InputGate unionedInputGate2 = InputGateUtil.createInputGate(inputGates2.toArray(new InputGate[0])); - - // create a Input instance for each input - CheckpointedInputGate[] checkpointedInputGates = InputProcessorUtil.createCheckpointedInputGatePair( - streamTask, - checkpointingMode, - ioManager, - unionedInputGate1, - unionedInputGate2, - taskManagerConfig, - taskName); - checkState(checkpointedInputGates.length == 2); this.input1 = new StreamTaskNetworkInput(checkpointedInputGates[0], inputSerializer1, ioManager, 0); this.input2 = new StreamTaskNetworkInput(checkpointedInputGates[1], inputSerializer2, ioManager, 1); + this.lock = checkNotNull(lock); + this.streamOperator = checkNotNull(streamOperator); + this.inputSelectionHandler = checkNotNull(inputSelectionHandler); + this.statusWatermarkValve1 = new StatusWatermarkValve( - unionedInputGate1.getNumberOfInputChannels(), + checkpointedInputGates[0].getNumberOfInputChannels(), new ForwardingValveOutputHandler(streamOperator, lock, streamStatusMaintainer, input1WatermarkGauge, 0)); this.statusWatermarkValve2 = new StatusWatermarkValve( - unionedInputGate2.getNumberOfInputChannels(), + checkpointedInputGates[1].getNumberOfInputChannels(), new ForwardingValveOutputHandler(streamOperator, lock, streamStatusMaintainer, input2WatermarkGauge, 1)); this.operatorChain = checkNotNull(operatorChain); this.numRecordsIn = checkNotNull(numRecordsIn); - - this.firstStatus = StreamStatus.ACTIVE; - this.secondStatus = StreamStatus.ACTIVE; - - this.lastReadInputIndex = 1; // always try to read from the first input - - this.isPrepared = false; } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java index 41fea75948..f8c96facbf 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java @@ -23,12 +23,17 @@ import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.streaming.api.operators.InputSelectable; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; +import org.apache.flink.streaming.runtime.io.CheckpointedInputGate; +import org.apache.flink.streaming.runtime.io.InputGateUtil; +import org.apache.flink.streaming.runtime.io.InputProcessorUtil; import org.apache.flink.streaming.runtime.io.StreamTwoInputProcessor; import org.apache.flink.streaming.runtime.io.TwoInputSelectionHandler; import java.io.IOException; import java.util.Collection; +import static org.apache.flink.util.Preconditions.checkState; + /** * A {@link StreamTask} for executing a {@link TwoInputStreamOperator} and supporting * the {@link TwoInputStreamOperator} to select input for reading. @@ -50,20 +55,31 @@ public class TwoInputStreamTask extends AbstractTwoInputStreamTas TwoInputSelectionHandler twoInputSelectionHandler = new TwoInputSelectionHandler( headOperator instanceof InputSelectable ? (InputSelectable) headOperator : null); - this.inputProcessor = new StreamTwoInputProcessor<>( - inputGates1, inputGates2, - inputDeserializer1, inputDeserializer2, + InputGate unionedInputGate1 = InputGateUtil.createInputGate(inputGates1.toArray(new InputGate[0])); + InputGate unionedInputGate2 = InputGateUtil.createInputGate(inputGates2.toArray(new InputGate[0])); + + // create a Input instance for each input + CheckpointedInputGate[] checkpointedInputGates = InputProcessorUtil.createCheckpointedInputGatePair( this, getConfiguration().getCheckpointMode(), - getCheckpointLock(), getEnvironment().getIOManager(), + unionedInputGate1, + unionedInputGate2, getEnvironment().getTaskManagerInfo().getConfiguration(), + getTaskNameWithSubtaskAndId()); + checkState(checkpointedInputGates.length == 2); + + inputProcessor = new StreamTwoInputProcessor<>( + checkpointedInputGates, + inputDeserializer1, + inputDeserializer2, + getCheckpointLock(), + getEnvironment().getIOManager(), getStreamStatusMaintainer(), - this.headOperator, + headOperator, twoInputSelectionHandler, input1WatermarkGauge, input2WatermarkGauge, - getTaskNameWithSubtaskAndId(), operatorChain, setupNumRecordsInCounter(headOperator)); } -- Gitee From 68386c434d21d2d1fac9ffed38fe5ac1fdee1045 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Tue, 10 Sep 2019 12:34:48 +0800 Subject: [PATCH 011/268] [hotfix][task] Refactor the constrcutor of StreamOneInputProcessor --- .../runtime/io/StreamOneInputProcessor.java | 33 +++---------------- .../runtime/tasks/OneInputStreamTask.java | 22 +++++++++---- 2 files changed, 21 insertions(+), 34 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java index 410e2be211..4d6e5bbafa 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java @@ -20,12 +20,8 @@ package org.apache.flink.streaming.runtime.io; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.network.partition.consumer.InputGate; -import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; -import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; @@ -35,7 +31,6 @@ import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve; import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import org.apache.flink.streaming.runtime.tasks.OperatorChain; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,7 +39,6 @@ import java.io.IOException; import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; /** * Input reader for {@link org.apache.flink.streaming.runtime.tasks.OneInputStreamTask}. @@ -86,44 +80,27 @@ public final class StreamOneInputProcessor implements StreamInputProcessor { private final Counter numRecordsIn; public StreamOneInputProcessor( - InputGate[] inputGates, + CheckpointedInputGate checkpointedInputGate, TypeSerializer inputSerializer, - StreamTask checkpointedTask, - CheckpointingMode checkpointMode, Object lock, IOManager ioManager, - Configuration taskManagerConfig, StreamStatusMaintainer streamStatusMaintainer, OneInputStreamOperator streamOperator, - TaskIOMetricGroup metrics, WatermarkGauge watermarkGauge, - String taskName, OperatorChain operatorChain, - Counter numRecordsIn) throws IOException { + Counter numRecordsIn) { - InputGate inputGate = InputGateUtil.createInputGate(inputGates); - - CheckpointedInputGate barrierHandler = InputProcessorUtil.createCheckpointedInputGate( - checkpointedTask, - checkpointMode, - ioManager, - inputGate, - taskManagerConfig, - taskName); - this.input = new StreamTaskNetworkInput(barrierHandler, inputSerializer, ioManager, 0); + this.input = new StreamTaskNetworkInput(checkpointedInputGate, inputSerializer, ioManager, 0); this.lock = checkNotNull(lock); - this.streamStatusMaintainer = checkNotNull(streamStatusMaintainer); this.streamOperator = checkNotNull(streamOperator); this.statusWatermarkValve = new StatusWatermarkValve( - inputGate.getNumberOfInputChannels(), + checkpointedInputGate.getNumberOfInputChannels(), new ForwardingValveOutputHandler(streamOperator, lock)); - this.watermarkGauge = watermarkGauge; - metrics.gauge("checkpointAlignmentTime", barrierHandler::getAlignmentDurationNanos); - + this.watermarkGauge = checkNotNull(watermarkGauge); this.operatorChain = checkNotNull(operatorChain); this.numRecordsIn = checkNotNull(numRecordsIn); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java index b7981b74a8..1bd86fbf66 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java @@ -24,8 +24,12 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.io.CheckpointedInputGate; +import org.apache.flink.streaming.runtime.io.InputGateUtil; +import org.apache.flink.streaming.runtime.io.InputProcessorUtil; import org.apache.flink.streaming.runtime.io.StreamOneInputProcessor; import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; @@ -74,20 +78,26 @@ public class OneInputStreamTask extends StreamTask 0) { InputGate[] inputGates = getEnvironment().getAllInputGates(); + InputGate inputGate = InputGateUtil.createInputGate(inputGates); + CheckpointedInputGate checkpointedInputGate = InputProcessorUtil.createCheckpointedInputGate( + this, + configuration.getCheckpointMode(), + getEnvironment().getIOManager(), + inputGate, + getEnvironment().getTaskManagerInfo().getConfiguration(), + getTaskNameWithSubtaskAndId()); + + TaskIOMetricGroup taskIOMetricGroup = getEnvironment().getMetricGroup().getIOMetricGroup(); + taskIOMetricGroup.gauge("checkpointAlignmentTime", checkpointedInputGate::getAlignmentDurationNanos); inputProcessor = new StreamOneInputProcessor<>( - inputGates, + checkpointedInputGate, inSerializer, - this, - configuration.getCheckpointMode(), getCheckpointLock(), getEnvironment().getIOManager(), - getEnvironment().getTaskManagerInfo().getConfiguration(), getStreamStatusMaintainer(), headOperator, - getEnvironment().getMetricGroup().getIOMetricGroup(), inputWatermarkGauge, - getTaskNameWithSubtaskAndId(), operatorChain, setupNumRecordsInCounter(headOperator)); } -- Gitee From a80bbf187b8a7318db3a8b117e7281eaa174c647 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Tue, 10 Sep 2019 13:22:08 +0800 Subject: [PATCH 012/268] [FLINK-13766][task] Refactor the implementation of StreamInputProcessor based on PushingAsyncDataInput#emitNext The current data processing in task input processor is based on the way of NullableAsyncDataInput#pollNext. In order to unify the processing stack for new source operator, we introduce the new PushingAsyncDataInput#emitNext(Output) instead. Then we need to adjust the existing implementations of StreamOneInputProcessor/StreamTwoInputProcessor based on this new way. To do so, we could integrate all the task inputs from network/source in a unified processor on runtime side. --- .../streaming/runtime/io/InputStatus.java | 47 ++++ .../runtime/io/PushingAsyncDataInput.java | 60 ++++++ .../runtime/io/StreamOneInputProcessor.java | 128 ++--------- .../streaming/runtime/io/StreamTaskInput.java | 10 +- .../runtime/io/StreamTaskNetworkInput.java | 52 +++-- .../runtime/io/StreamTwoInputProcessor.java | 203 +++++++++--------- .../streamstatus/StatusWatermarkValve.java | 38 ++-- .../runtime/tasks/OneInputStreamTask.java | 139 ++++++++++-- .../io/StreamTaskNetworkInputTest.java | 63 +++++- .../StatusWatermarkValveTest.java | 47 ++-- 10 files changed, 482 insertions(+), 305 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputStatus.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/PushingAsyncDataInput.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputStatus.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputStatus.java new file mode 100644 index 0000000000..a2d8d29a41 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputStatus.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.streaming.runtime.io; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput.DataOutput; + +/** + * An {@link InputStatus} indicates one input state which might be currently + * available, not available or already finished. It is returned while calling + * {@link PushingAsyncDataInput#emitNext(DataOutput)}. + */ +@Internal +public enum InputStatus { + + /** + * Indicator that more data is available and the input can be called immediately again + * to emit more data. + */ + MORE_AVAILABLE, + + /** + * Indicator that no data is currently available, but more data will be available in the + * future again. + */ + NOTHING_AVAILABLE, + + /** + * Indicator that the input has reached the end of data. + */ + END_OF_INPUT +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/PushingAsyncDataInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/PushingAsyncDataInput.java new file mode 100644 index 0000000000..b912370ce0 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/PushingAsyncDataInput.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.streaming.runtime.io; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.io.AvailabilityListener; +import org.apache.flink.runtime.io.PullingAsyncDataInput; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; + +/** + * The variant of {@link PullingAsyncDataInput} that is defined for handling both network + * input and source input in a unified way via {@link #emitNext(DataOutput)} instead + * of returning {@code Optional.empty()} via {@link PullingAsyncDataInput#pollNext()}. + */ +@Internal +public interface PushingAsyncDataInput extends AvailabilityListener { + + /** + * Pushes the next element to the output from current data input, and returns + * the input status to indicate whether there are more available data in + * current input. + * + *

This method should be non blocking. + */ + InputStatus emitNext(DataOutput output) throws Exception; + + /** + * Basic data output interface used in emitting the next element from data input. + * + * @param The type encapsulated with the stream record. + */ + interface DataOutput { + + void emitRecord(StreamRecord streamRecord) throws Exception; + + void emitWatermark(Watermark watermark) throws Exception; + + void emitStreamStatus(StreamStatus streamStatus) throws Exception; + + void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java index 4d6e5bbafa..4c21791ff5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java @@ -19,17 +19,7 @@ package org.apache.flink.streaming.runtime.io; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.metrics.Counter; -import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; -import org.apache.flink.streaming.runtime.streamrecord.StreamElement; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve; -import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; -import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput.DataOutput; import org.apache.flink.streaming.runtime.tasks.OperatorChain; import org.slf4j.Logger; @@ -43,15 +33,6 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** * Input reader for {@link org.apache.flink.streaming.runtime.tasks.OneInputStreamTask}. * - *

This internally uses a {@link StatusWatermarkValve} to keep track of {@link Watermark} and - * {@link StreamStatus} events, and forwards them to event subscribers once the - * {@link StatusWatermarkValve} determines the {@link Watermark} from all inputs has advanced, or - * that a {@link StreamStatus} needs to be propagated downstream to denote a status change. - * - *

Forwarding elements, watermarks, or status status elements must be protected by synchronizing - * on the given lock object. This ensures that we don't call methods on a - * {@link OneInputStreamOperator} concurrently with the timer callback or other things. - * * @param The type of the record that can be read with this record reader. */ @Internal @@ -59,50 +40,23 @@ public final class StreamOneInputProcessor implements StreamInputProcessor { private static final Logger LOG = LoggerFactory.getLogger(StreamOneInputProcessor.class); - private final StreamTaskInput input; + private final StreamTaskInput input; + private final DataOutput output; private final Object lock; private final OperatorChain operatorChain; - // ---------------- Status and Watermark Valve ------------------ - - /** Valve that controls how watermarks and stream statuses are forwarded. */ - private StatusWatermarkValve statusWatermarkValve; - - private final StreamStatusMaintainer streamStatusMaintainer; - - private final OneInputStreamOperator streamOperator; - - // ---------------- Metrics ------------------ - - private final WatermarkGauge watermarkGauge; - private final Counter numRecordsIn; - public StreamOneInputProcessor( - CheckpointedInputGate checkpointedInputGate, - TypeSerializer inputSerializer, + StreamTaskInput input, + DataOutput output, Object lock, - IOManager ioManager, - StreamStatusMaintainer streamStatusMaintainer, - OneInputStreamOperator streamOperator, - WatermarkGauge watermarkGauge, - OperatorChain operatorChain, - Counter numRecordsIn) { - - this.input = new StreamTaskNetworkInput(checkpointedInputGate, inputSerializer, ioManager, 0); + OperatorChain operatorChain) { + this.input = checkNotNull(input); + this.output = checkNotNull(output); this.lock = checkNotNull(lock); - this.streamStatusMaintainer = checkNotNull(streamStatusMaintainer); - this.streamOperator = checkNotNull(streamOperator); - - this.statusWatermarkValve = new StatusWatermarkValve( - checkpointedInputGate.getNumberOfInputChannels(), - new ForwardingValveOutputHandler(streamOperator, lock)); - - this.watermarkGauge = checkNotNull(watermarkGauge); this.operatorChain = checkNotNull(operatorChain); - this.numRecordsIn = checkNotNull(numRecordsIn); } @Override @@ -117,77 +71,19 @@ public final class StreamOneInputProcessor implements StreamInputProcessor { @Override public boolean processInput() throws Exception { - StreamElement recordOrMark = input.pollNextNullable(); - if (recordOrMark != null) { - processElement(recordOrMark, input.getLastChannel()); - } - checkFinished(); - - return recordOrMark != null; - } + InputStatus status = input.emitNext(output); - private void processElement(StreamElement recordOrMark, int channel) throws Exception { - if (recordOrMark.isRecord()) { - // now we can do the actual processing - StreamRecord record = recordOrMark.asRecord(); - synchronized (lock) { - numRecordsIn.inc(); - streamOperator.setKeyContextElement1(record); - streamOperator.processElement(record); - } - } - else if (recordOrMark.isWatermark()) { - // handle watermark - statusWatermarkValve.inputWatermark(recordOrMark.asWatermark(), channel); - } else if (recordOrMark.isStreamStatus()) { - // handle stream status - statusWatermarkValve.inputStreamStatus(recordOrMark.asStreamStatus(), channel); - } else if (recordOrMark.isLatencyMarker()) { - // handle latency marker - synchronized (lock) { - streamOperator.processLatencyMarker(recordOrMark.asLatencyMarker()); - } - } else { - throw new UnsupportedOperationException("Unknown type of StreamElement"); - } - } - - private void checkFinished() throws Exception { - if (input.isFinished()) { + if (status == InputStatus.END_OF_INPUT) { synchronized (lock) { operatorChain.endInput(1); } } + + return status == InputStatus.MORE_AVAILABLE; } @Override public void close() throws IOException { input.close(); } - - private class ForwardingValveOutputHandler implements StatusWatermarkValve.ValveOutputHandler { - private final OneInputStreamOperator operator; - private final Object lock; - - private ForwardingValveOutputHandler(final OneInputStreamOperator operator, final Object lock) { - this.operator = checkNotNull(operator); - this.lock = checkNotNull(lock); - } - - @Override - public void handleWatermark(Watermark watermark) throws Exception { - synchronized (lock) { - watermarkGauge.setCurrentWatermark(watermark.getTimestamp()); - operator.processWatermark(watermark); - } - } - - @SuppressWarnings("unchecked") - @Override - public void handleStreamStatus(StreamStatus streamStatus) { - synchronized (lock) { - streamStatusMaintainer.toggleStreamStatus(streamStatus); - } - } - } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskInput.java index 19fd76513f..484d9cd8d8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskInput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskInput.java @@ -18,8 +18,6 @@ package org.apache.flink.streaming.runtime.io; import org.apache.flink.annotation.Internal; -import org.apache.flink.runtime.io.NullableAsyncDataInput; -import org.apache.flink.streaming.runtime.streamrecord.StreamElement; import java.io.Closeable; @@ -27,15 +25,9 @@ import java.io.Closeable; * Basic interface for inputs of stream operators. */ @Internal -public interface StreamTaskInput extends NullableAsyncDataInput, Closeable { +public interface StreamTaskInput extends PushingAsyncDataInput, Closeable { int UNSPECIFIED = -1; - /** - * @return channel index of last returned {@link StreamElement}, or {@link #UNSPECIFIED} if - * it is unspecified. - */ - int getLastChannel(); - /** * Returns the input index of this input. */ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java index ff592629f8..8f85709f26 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java @@ -31,22 +31,33 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; +import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamElement; import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; - -import javax.annotation.Nullable; +import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve; +import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; import java.io.IOException; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; /** * Implementation of {@link StreamTaskInput} that wraps an input from network taken from {@link CheckpointedInputGate}. + * + *

This internally uses a {@link StatusWatermarkValve} to keep track of {@link Watermark} and + * {@link StreamStatus} events, and forwards them to event subscribers once the + * {@link StatusWatermarkValve} determines the {@link Watermark} from all inputs has advanced, or + * that a {@link StreamStatus} needs to be propagated downstream to denote a status change. + * + *

Forwarding elements, watermarks, or status status elements must be protected by synchronizing + * on the given lock object. This ensures that we don't call methods on a + * {@link StreamInputProcessor} concurrently with the timer callback or other things. */ @Internal -public final class StreamTaskNetworkInput implements StreamTaskInput { +public final class StreamTaskNetworkInput implements StreamTaskInput { private final CheckpointedInputGate checkpointedInputGate; @@ -54,6 +65,9 @@ public final class StreamTaskNetworkInput implements StreamTaskInput { private final RecordDeserializer>[] recordDeserializers; + /** Valve that controls how watermarks and stream statuses are forwarded. */ + private final StatusWatermarkValve statusWatermarkValve; + private final int inputIndex; private int lastChannel = UNSPECIFIED; @@ -67,6 +81,7 @@ public final class StreamTaskNetworkInput implements StreamTaskInput { CheckpointedInputGate checkpointedInputGate, TypeSerializer inputSerializer, IOManager ioManager, + StatusWatermarkValve statusWatermarkValve, int inputIndex) { this.checkpointedInputGate = checkpointedInputGate; this.deserializationDelegate = new NonReusingDeserializationDelegate<>( @@ -79,6 +94,7 @@ public final class StreamTaskNetworkInput implements StreamTaskInput { ioManager.getSpillingDirectoriesPaths()); } + this.statusWatermarkValve = checkNotNull(statusWatermarkValve); this.inputIndex = inputIndex; } @@ -86,6 +102,7 @@ public final class StreamTaskNetworkInput implements StreamTaskInput { StreamTaskNetworkInput( CheckpointedInputGate checkpointedInputGate, TypeSerializer inputSerializer, + StatusWatermarkValve statusWatermarkValve, int inputIndex, RecordDeserializer>[] recordDeserializers) { @@ -93,12 +110,12 @@ public final class StreamTaskNetworkInput implements StreamTaskInput { this.deserializationDelegate = new NonReusingDeserializationDelegate<>( new StreamElementSerializer<>(inputSerializer)); this.recordDeserializers = recordDeserializers; + this.statusWatermarkValve = statusWatermarkValve; this.inputIndex = inputIndex; } @Override - @Nullable - public StreamElement pollNextNullable() throws Exception { + public InputStatus emitNext(DataOutput output) throws Exception { while (true) { // get the stream element from the deserializer @@ -110,7 +127,8 @@ public final class StreamTaskNetworkInput implements StreamTaskInput { } if (result.isFullRecord()) { - return deserializationDelegate.getInstance(); + processElement(deserializationDelegate.getInstance(), output); + return InputStatus.MORE_AVAILABLE; } } @@ -124,12 +142,27 @@ public final class StreamTaskNetworkInput implements StreamTaskInput { if (!checkpointedInputGate.isEmpty()) { throw new IllegalStateException("Trailing data in checkpoint barrier handler."); } + return InputStatus.END_OF_INPUT; } - return null; + return InputStatus.NOTHING_AVAILABLE; } } } + private void processElement(StreamElement recordOrMark, DataOutput output) throws Exception { + if (recordOrMark.isRecord()){ + output.emitRecord(recordOrMark.asRecord()); + } else if (recordOrMark.isWatermark()) { + statusWatermarkValve.inputWatermark(recordOrMark.asWatermark(), lastChannel); + } else if (recordOrMark.isLatencyMarker()) { + output.emitLatencyMarker(recordOrMark.asLatencyMarker()); + } else if (recordOrMark.isStreamStatus()) { + statusWatermarkValve.inputStreamStatus(recordOrMark.asStreamStatus(), lastChannel); + } else { + throw new UnsupportedOperationException("Unknown type of StreamElement"); + } + } + private void processBufferOrEvent(BufferOrEvent bufferOrEvent) throws IOException { if (bufferOrEvent.isBuffer()) { lastChannel = bufferOrEvent.getChannelIndex(); @@ -154,11 +187,6 @@ public final class StreamTaskNetworkInput implements StreamTaskInput { } } - @Override - public int getLastChannel() { - return lastChannel; - } - @Override public int getInputIndex() { return inputIndex; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index 4541fcbb62..1e5d31fe88 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -24,8 +24,9 @@ import org.apache.flink.metrics.Counter; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput.DataOutput; import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; -import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve; import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; @@ -33,6 +34,7 @@ import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import org.apache.flink.streaming.runtime.tasks.OperatorChain; import org.apache.flink.streaming.runtime.tasks.TwoInputStreamTask; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.function.ThrowingConsumer; import java.io.IOException; import java.util.concurrent.CompletableFuture; @@ -48,22 +50,17 @@ import static org.apache.flink.util.Preconditions.checkNotNull; @Internal public final class StreamTwoInputProcessor implements StreamInputProcessor { - private final TwoInputStreamOperator streamOperator; - private final TwoInputSelectionHandler inputSelectionHandler; private final Object lock; - private final StreamTaskInput input1; - private final StreamTaskInput input2; + private final StreamTaskInput input1; + private final StreamTaskInput input2; private final OperatorChain operatorChain; - /** - * Valves that control how watermarks and stream statuses from the 2 inputs are forwarded. - */ - private final StatusWatermarkValve statusWatermarkValve1; - private final StatusWatermarkValve statusWatermarkValve2; + private final DataOutput output1; + private final DataOutput output2; /** * Stream status for the two inputs. We need to keep track for determining when @@ -75,8 +72,6 @@ public final class StreamTwoInputProcessor implements StreamInputProce /** Always try to read from the first input. */ private int lastReadInputIndex = 1; - private final Counter numRecordsIn; - private boolean isPrepared; public StreamTwoInputProcessor( @@ -93,22 +88,63 @@ public final class StreamTwoInputProcessor implements StreamInputProce OperatorChain operatorChain, Counter numRecordsIn) { - this.input1 = new StreamTaskNetworkInput(checkpointedInputGates[0], inputSerializer1, ioManager, 0); - this.input2 = new StreamTaskNetworkInput(checkpointedInputGates[1], inputSerializer2, ioManager, 1); - this.lock = checkNotNull(lock); - this.streamOperator = checkNotNull(streamOperator); this.inputSelectionHandler = checkNotNull(inputSelectionHandler); - this.statusWatermarkValve1 = new StatusWatermarkValve( - checkpointedInputGates[0].getNumberOfInputChannels(), - new ForwardingValveOutputHandler(streamOperator, lock, streamStatusMaintainer, input1WatermarkGauge, 0)); - this.statusWatermarkValve2 = new StatusWatermarkValve( - checkpointedInputGates[1].getNumberOfInputChannels(), - new ForwardingValveOutputHandler(streamOperator, lock, streamStatusMaintainer, input2WatermarkGauge, 1)); + this.output1 = new StreamTaskNetworkOutput<>( + streamOperator, + record -> processRecord1(record, streamOperator, numRecordsIn), + lock, + streamStatusMaintainer, + input1WatermarkGauge, + 0); + this.output2 = new StreamTaskNetworkOutput<>( + streamOperator, + record -> processRecord2(record, streamOperator, numRecordsIn), + lock, + streamStatusMaintainer, + input2WatermarkGauge, + 1); + + this.input1 = new StreamTaskNetworkInput<>( + checkpointedInputGates[0], + inputSerializer1, + ioManager, + new StatusWatermarkValve(checkpointedInputGates[0].getNumberOfInputChannels(), output1), + 0); + this.input2 = new StreamTaskNetworkInput<>( + checkpointedInputGates[1], + inputSerializer2, + ioManager, + new StatusWatermarkValve(checkpointedInputGates[1].getNumberOfInputChannels(), output2), + 1); this.operatorChain = checkNotNull(operatorChain); - this.numRecordsIn = checkNotNull(numRecordsIn); + } + + private void processRecord1( + StreamRecord record, + TwoInputStreamOperator streamOperator, + Counter numRecordsIn) throws Exception { + + streamOperator.setKeyContextElement1(record); + streamOperator.processElement1(record); + postProcessRecord(numRecordsIn); + } + + private void processRecord2( + StreamRecord record, + TwoInputStreamOperator streamOperator, + Counter numRecordsIn) throws Exception { + + streamOperator.setKeyContextElement2(record); + streamOperator.processElement2(record); + postProcessRecord(numRecordsIn); + } + + private void postProcessRecord(Counter numRecordsIn) { + numRecordsIn.inc(); + inputSelectionHandler.nextSelection(); } @Override @@ -140,30 +176,23 @@ public final class StreamTwoInputProcessor implements StreamInputProce } lastReadInputIndex = readingInputIndex; - StreamElement recordOrMark; + InputStatus status; if (readingInputIndex == 0) { - recordOrMark = input1.pollNextNullable(); - if (recordOrMark != null) { - processElement1(recordOrMark, input1.getLastChannel()); - } - checkFinished(input1, lastReadInputIndex); + status = input1.emitNext(output1); } else { - recordOrMark = input2.pollNextNullable(); - if (recordOrMark != null) { - processElement2(recordOrMark, input2.getLastChannel()); - } - checkFinished(input2, lastReadInputIndex); + status = input2.emitNext(output2); } + checkFinished(status, lastReadInputIndex); - if (recordOrMark == null) { + if (status != InputStatus.MORE_AVAILABLE) { inputSelectionHandler.setUnavailableInput(readingInputIndex); } - return recordOrMark != null; + return status == InputStatus.MORE_AVAILABLE; } - private void checkFinished(StreamTaskInput input, int inputIndex) throws Exception { - if (input.isFinished()) { + private void checkFinished(InputStatus status, int inputIndex) throws Exception { + if (status == InputStatus.END_OF_INPUT) { synchronized (lock) { operatorChain.endInput(getInputId(inputIndex)); inputSelectionHandler.nextSelection(); @@ -232,52 +261,6 @@ public final class StreamTwoInputProcessor implements StreamInputProce } } - private void processElement1(StreamElement recordOrMark, int channel) throws Exception { - if (recordOrMark.isRecord()) { - StreamRecord record = recordOrMark.asRecord(); - synchronized (lock) { - numRecordsIn.inc(); - streamOperator.setKeyContextElement1(record); - streamOperator.processElement1(record); - inputSelectionHandler.nextSelection(); - } - } - else if (recordOrMark.isWatermark()) { - statusWatermarkValve1.inputWatermark(recordOrMark.asWatermark(), channel); - } else if (recordOrMark.isStreamStatus()) { - statusWatermarkValve1.inputStreamStatus(recordOrMark.asStreamStatus(), channel); - } else if (recordOrMark.isLatencyMarker()) { - synchronized (lock) { - streamOperator.processLatencyMarker1(recordOrMark.asLatencyMarker()); - } - } else { - throw new UnsupportedOperationException("Unknown type of StreamElement on input1"); - } - } - - private void processElement2(StreamElement recordOrMark, int channel) throws Exception { - if (recordOrMark.isRecord()) { - StreamRecord record = recordOrMark.asRecord(); - synchronized (lock) { - numRecordsIn.inc(); - streamOperator.setKeyContextElement2(record); - streamOperator.processElement2(record); - inputSelectionHandler.nextSelection(); - } - } - else if (recordOrMark.isWatermark()) { - statusWatermarkValve2.inputWatermark(recordOrMark.asWatermark(), channel); - } else if (recordOrMark.isStreamStatus()) { - statusWatermarkValve2.inputStreamStatus(recordOrMark.asStreamStatus(), channel); - } else if (recordOrMark.isLatencyMarker()) { - synchronized (lock) { - streamOperator.processLatencyMarker2(recordOrMark.asLatencyMarker()); - } - } else { - throw new UnsupportedOperationException("Unknown type of StreamElement on input2"); - } - } - private void prepareForProcessing() { // Note: the first call to nextSelection () on the operator must be made after this operator // is opened to ensure that any changes about the input selection in its open() @@ -318,37 +301,52 @@ public final class StreamTwoInputProcessor implements StreamInputProce return inputIndex + 1; } - private class ForwardingValveOutputHandler implements StatusWatermarkValve.ValveOutputHandler { + /** + * The network data output implementation used for processing stream elements + * from {@link StreamTaskNetworkInput} in two input selective processor. + */ + private class StreamTaskNetworkOutput implements DataOutput { private final TwoInputStreamOperator operator; + /** The function way is only used for frequent record processing as for JIT optimization. */ + private final ThrowingConsumer, Exception> recordConsumer; + private final Object lock; + /** The maintainer toggles the current stream status as well as retrieves it. */ private final StreamStatusMaintainer streamStatusMaintainer; private final WatermarkGauge inputWatermarkGauge; + /** The input index to indicate how to process elements by two input operator. */ private final int inputIndex; - private ForwardingValveOutputHandler( - TwoInputStreamOperator operator, - Object lock, - StreamStatusMaintainer streamStatusMaintainer, - WatermarkGauge inputWatermarkGauge, - int inputIndex) { + private StreamTaskNetworkOutput( + TwoInputStreamOperator operator, + ThrowingConsumer, Exception> recordConsumer, + Object lock, + StreamStatusMaintainer streamStatusMaintainer, + WatermarkGauge inputWatermarkGauge, + int inputIndex) { this.operator = checkNotNull(operator); + this.recordConsumer = checkNotNull(recordConsumer); this.lock = checkNotNull(lock); - this.streamStatusMaintainer = checkNotNull(streamStatusMaintainer); - - this.inputWatermarkGauge = inputWatermarkGauge; - + this.inputWatermarkGauge = checkNotNull(inputWatermarkGauge); this.inputIndex = inputIndex; } @Override - public void handleWatermark(Watermark watermark) throws Exception { + public void emitRecord(StreamRecord record) throws Exception { + synchronized (lock) { + recordConsumer.accept(record); + } + } + + @Override + public void emitWatermark(Watermark watermark) throws Exception { synchronized (lock) { inputWatermarkGauge.setCurrentWatermark(watermark.getTimestamp()); if (inputIndex == 0) { @@ -360,7 +358,7 @@ public final class StreamTwoInputProcessor implements StreamInputProce } @Override - public void handleStreamStatus(StreamStatus streamStatus) { + public void emitStreamStatus(StreamStatus streamStatus) { synchronized (lock) { final StreamStatus anotherStreamStatus; if (inputIndex == 0) { @@ -383,5 +381,16 @@ public final class StreamTwoInputProcessor implements StreamInputProce } } } + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception { + synchronized (lock) { + if (inputIndex == 0) { + operator.processLatencyMarker1(latencyMarker); + } else { + operator.processLatencyMarker2(latencyMarker); + } + } + } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamstatus/StatusWatermarkValve.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamstatus/StatusWatermarkValve.java index e69d41aedb..43704d5706 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamstatus/StatusWatermarkValve.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamstatus/StatusWatermarkValve.java @@ -21,6 +21,7 @@ package org.apache.flink.streaming.runtime.streamstatus; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput.DataOutput; import org.apache.flink.util.Preconditions; import static org.apache.flink.util.Preconditions.checkArgument; @@ -29,24 +30,13 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** * A {@code StatusWatermarkValve} embodies the logic of how {@link Watermark} and {@link StreamStatus} are propagated to * downstream outputs, given a set of one or multiple input channels that continuously receive them. Usages of this - * class need to define the number of input channels that the valve needs to handle, as well as provide a customized - * implementation of {@link ValveOutputHandler}, which is called by the valve only when it determines a new watermark or - * stream status can be propagated. + * class need to define the number of input channels that the valve needs to handle, as well as provide a implementation of + * {@link DataOutput}, which is called by the valve only when it determines a new watermark or stream status can be propagated. */ @Internal public class StatusWatermarkValve { - /** - * Usages of {@code StatusWatermarkValve} should implement a {@code ValveOutputHandler} - * to handle watermark and stream status outputs from the valve. - */ - public interface ValveOutputHandler { - void handleWatermark(Watermark watermark) throws Exception; - - void handleStreamStatus(StreamStatus streamStatus) throws Exception; - } - - private final ValveOutputHandler outputHandler; + private final DataOutput output; // ------------------------------------------------------------------------ // Runtime state for watermark & stream status output determination @@ -68,9 +58,9 @@ public class StatusWatermarkValve { * Returns a new {@code StatusWatermarkValve}. * * @param numInputChannels the number of input channels that this valve will need to handle - * @param outputHandler the customized output handler for the valve + * @param output the customized output handler for the valve */ - public StatusWatermarkValve(int numInputChannels, ValveOutputHandler outputHandler) { + public StatusWatermarkValve(int numInputChannels, DataOutput output) { checkArgument(numInputChannels > 0); this.channelStatuses = new InputChannelStatus[numInputChannels]; for (int i = 0; i < numInputChannels; i++) { @@ -80,7 +70,7 @@ public class StatusWatermarkValve { channelStatuses[i].isWatermarkAligned = true; } - this.outputHandler = checkNotNull(outputHandler); + this.output = checkNotNull(output); this.lastOutputWatermark = Long.MIN_VALUE; this.lastOutputStreamStatus = StreamStatus.ACTIVE; @@ -88,7 +78,7 @@ public class StatusWatermarkValve { /** * Feed a {@link Watermark} into the valve. If the input triggers the valve to output a new Watermark, - * {@link ValveOutputHandler#handleWatermark(Watermark)} will be called to process the new Watermark. + * {@link DataOutput#emitWatermark(Watermark)} will be called to process the new Watermark. * * @param watermark the watermark to feed to the valve * @param channelIndex the index of the channel that the fed watermark belongs to (index starting from 0) @@ -115,8 +105,8 @@ public class StatusWatermarkValve { /** * Feed a {@link StreamStatus} into the valve. This may trigger the valve to output either a new Stream Status, - * for which {@link ValveOutputHandler#handleStreamStatus(StreamStatus)} will be called, or a new Watermark, - * for which {@link ValveOutputHandler#handleWatermark(Watermark)} will be called. + * for which {@link DataOutput#emitStreamStatus(StreamStatus)} will be called, or a new Watermark, + * for which {@link DataOutput#emitWatermark(Watermark)} will be called. * * @param streamStatus the stream status to feed to the valve * @param channelIndex the index of the channel that the fed stream status belongs to (index starting from 0) @@ -144,7 +134,7 @@ public class StatusWatermarkValve { } lastOutputStreamStatus = StreamStatus.IDLE; - outputHandler.handleStreamStatus(lastOutputStreamStatus); + output.emitStreamStatus(lastOutputStreamStatus); } else if (channelStatuses[channelIndex].watermark == lastOutputWatermark) { // if the watermark of the channel that just became idle equals the last output // watermark (the previous overall min watermark), we may be able to find a new @@ -165,7 +155,7 @@ public class StatusWatermarkValve { // status because at least one of the input channels is now active if (lastOutputStreamStatus.isIdle()) { lastOutputStreamStatus = StreamStatus.ACTIVE; - outputHandler.handleStreamStatus(lastOutputStreamStatus); + output.emitStreamStatus(lastOutputStreamStatus); } } } @@ -186,7 +176,7 @@ public class StatusWatermarkValve { // from some remaining aligned channel, and is also larger than the last output watermark if (hasAlignedChannels && newMinWatermark > lastOutputWatermark) { lastOutputWatermark = newMinWatermark; - outputHandler.handleWatermark(new Watermark(lastOutputWatermark)); + output.emitWatermark(new Watermark(lastOutputWatermark)); } } @@ -199,7 +189,7 @@ public class StatusWatermarkValve { if (maxWatermark > lastOutputWatermark) { lastOutputWatermark = maxWatermark; - outputHandler.handleWatermark(new Watermark(lastOutputWatermark)); + output.emitWatermark(new Watermark(lastOutputWatermark)); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java index 1bd86fbf66..458cd06b2e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java @@ -21,20 +21,34 @@ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.metrics.Counter; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.CheckpointedInputGate; import org.apache.flink.streaming.runtime.io.InputGateUtil; import org.apache.flink.streaming.runtime.io.InputProcessorUtil; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput.DataOutput; import org.apache.flink.streaming.runtime.io.StreamOneInputProcessor; +import org.apache.flink.streaming.runtime.io.StreamTaskInput; +import org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput; import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve; +import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; +import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import javax.annotation.Nullable; +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * A {@link StreamTask} for executing a {@link OneInputStreamOperator}. */ @@ -72,37 +86,120 @@ public class OneInputStreamTask extends StreamTask inSerializer = configuration.getTypeSerializerIn1(getUserCodeClassLoader()); int numberOfInputs = configuration.getNumberOfInputs(); if (numberOfInputs > 0) { - InputGate[] inputGates = getEnvironment().getAllInputGates(); - InputGate inputGate = InputGateUtil.createInputGate(inputGates); - CheckpointedInputGate checkpointedInputGate = InputProcessorUtil.createCheckpointedInputGate( - this, - configuration.getCheckpointMode(), - getEnvironment().getIOManager(), - inputGate, - getEnvironment().getTaskManagerInfo().getConfiguration(), - getTaskNameWithSubtaskAndId()); - + CheckpointedInputGate inputGate = createCheckpointedInputGate(); TaskIOMetricGroup taskIOMetricGroup = getEnvironment().getMetricGroup().getIOMetricGroup(); - taskIOMetricGroup.gauge("checkpointAlignmentTime", checkpointedInputGate::getAlignmentDurationNanos); + taskIOMetricGroup.gauge("checkpointAlignmentTime", inputGate::getAlignmentDurationNanos); + DataOutput output = createDataOutput(); + StreamTaskInput input = createTaskInput(inputGate, output); inputProcessor = new StreamOneInputProcessor<>( - checkpointedInputGate, - inSerializer, + input, + output, getCheckpointLock(), - getEnvironment().getIOManager(), - getStreamStatusMaintainer(), - headOperator, - inputWatermarkGauge, - operatorChain, - setupNumRecordsInCounter(headOperator)); + operatorChain); } headOperator.getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_WATERMARK, this.inputWatermarkGauge); // wrap watermark gauge since registered metrics must be unique getEnvironment().getMetricGroup().gauge(MetricNames.IO_CURRENT_INPUT_WATERMARK, this.inputWatermarkGauge::getValue); } + + private CheckpointedInputGate createCheckpointedInputGate() throws IOException { + InputGate[] inputGates = getEnvironment().getAllInputGates(); + InputGate inputGate = InputGateUtil.createInputGate(inputGates); + + return InputProcessorUtil.createCheckpointedInputGate( + this, + configuration.getCheckpointMode(), + getEnvironment().getIOManager(), + inputGate, + getEnvironment().getTaskManagerInfo().getConfiguration(), + getTaskNameWithSubtaskAndId()); + } + + private DataOutput createDataOutput() { + return new StreamTaskNetworkOutput<>( + headOperator, + getStreamStatusMaintainer(), + getCheckpointLock(), + inputWatermarkGauge, + setupNumRecordsInCounter(headOperator)); + } + + private StreamTaskInput createTaskInput(CheckpointedInputGate inputGate, DataOutput output) { + int numberOfInputChannels = inputGate.getNumberOfInputChannels(); + StatusWatermarkValve statusWatermarkValve = new StatusWatermarkValve(numberOfInputChannels, output); + + TypeSerializer inSerializer = configuration.getTypeSerializerIn1(getUserCodeClassLoader()); + return new StreamTaskNetworkInput<>( + inputGate, + inSerializer, + getEnvironment().getIOManager(), + statusWatermarkValve, + 0); + } + + /** + * The network data output implementation used for processing stream elements + * from {@link StreamTaskNetworkInput} in one input processor. + */ + private static class StreamTaskNetworkOutput implements DataOutput { + + private final OneInputStreamOperator operator; + + /** The maintainer toggles the current stream status. */ + private final StreamStatusMaintainer streamStatusMaintainer; + + private final Object lock; + + private final WatermarkGauge watermarkGauge; + private final Counter numRecordsIn; + + private StreamTaskNetworkOutput( + OneInputStreamOperator operator, + StreamStatusMaintainer streamStatusMaintainer, + Object lock, + WatermarkGauge watermarkGauge, + Counter numRecordsIn) { + + this.operator = checkNotNull(operator); + this.streamStatusMaintainer = checkNotNull(streamStatusMaintainer); + this.lock = checkNotNull(lock); + this.watermarkGauge = checkNotNull(watermarkGauge); + this.numRecordsIn = checkNotNull(numRecordsIn); + } + + @Override + public void emitRecord(StreamRecord record) throws Exception { + synchronized (lock) { + numRecordsIn.inc(); + operator.setKeyContextElement1(record); + operator.processElement(record); + } + } + + @Override + public void emitWatermark(Watermark watermark) throws Exception { + synchronized (lock) { + watermarkGauge.setCurrentWatermark(watermark.getTimestamp()); + operator.processWatermark(watermark); + } + } + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception { + synchronized (lock) { + operator.processLatencyMarker(latencyMarker); + } + } + + @Override + public void emitStreamStatus(StreamStatus streamStatus) { + synchronized (lock) { + streamStatusMaintainer.toggleStreamStatus(streamStatus); + } + } + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java index cdc70c5962..17b9e063e8 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java @@ -32,9 +32,14 @@ import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate; import org.apache.flink.runtime.plugable.DeserializationDelegate; import org.apache.flink.runtime.plugable.SerializationDelegate; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput.DataOutput; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamElement; import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve; +import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; import org.junit.After; import org.junit.Test; @@ -44,9 +49,12 @@ import java.util.Arrays; import java.util.Collections; 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.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -74,17 +82,20 @@ public class StreamTaskNetworkInputTest { List buffers = Collections.singletonList(new BufferOrEvent(buffer, 0, false)); - StreamTaskNetworkInput input = new StreamTaskNetworkInput( + VerifyRecordsDataOutput output = new VerifyRecordsDataOutput<>(); + StreamTaskNetworkInput input = new StreamTaskNetworkInput<>( new CheckpointedInputGate( new MockInputGate(1, buffers, false), new EmptyBufferStorage(), new CheckpointBarrierTracker(1)), LongSerializer.INSTANCE, ioManager, + new StatusWatermarkValve(1, output), 0); - assertHasNextElement(input); - assertHasNextElement(input); + assertHasNextElement(input, output); + assertHasNextElement(input, output); + assertEquals(2, output.getNumberOfEmittedRecords()); } @Test @@ -101,19 +112,21 @@ public class StreamTaskNetworkInputTest { } TestRecordDeserializer[] copiedDeserializers = Arrays.copyOf(deserializers, deserializers.length); - StreamTaskNetworkInput input = new StreamTaskNetworkInput( + DataOutput output = new NoOpDataOutput<>(); + StreamTaskNetworkInput input = new StreamTaskNetworkInput<>( new CheckpointedInputGate( inputGate.getInputGate(), new EmptyBufferStorage(), new CheckpointBarrierTracker(1)), inSerializer, + new StatusWatermarkValve(1, output), 0, deserializers); for (int i = 0; i < numInputChannels; i++) { assertNotNull(deserializers[i]); inputGate.sendEvent(EndOfPartitionEvent.INSTANCE, i); - input.pollNextNullable(); + input.emitNext(output); assertNull(deserializers[i]); assertTrue(copiedDeserializers[i].isCleared()); } @@ -130,11 +143,10 @@ public class StreamTaskNetworkInputTest { assertFalse(serializer.copyToBufferBuilder(bufferBuilder).isFullBuffer()); } - private static void assertHasNextElement(StreamTaskNetworkInput input) throws Exception { + private static void assertHasNextElement(StreamTaskNetworkInput input, DataOutput output) throws Exception { assertTrue(input.isAvailable().isDone()); - StreamElement element = input.pollNextNullable(); - assertNotNull(element); - assertTrue(element.isRecord()); + InputStatus status = input.emitNext(output); + assertThat(status, is(InputStatus.MORE_AVAILABLE)); } private static class TestRecordDeserializer @@ -155,4 +167,37 @@ public class StreamTaskNetworkInputTest { return cleared; } } + + private static class NoOpDataOutput implements DataOutput { + + @Override + public void emitRecord(StreamRecord record) { + } + + @Override + public void emitWatermark(Watermark watermark) { + } + + @Override + public void emitStreamStatus(StreamStatus streamStatus) { + } + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) { + } + } + + private static class VerifyRecordsDataOutput extends NoOpDataOutput { + + private int numberOfEmittedRecords; + + @Override + public void emitRecord(StreamRecord record) { + numberOfEmittedRecords++; + } + + int getNumberOfEmittedRecords() { + return numberOfEmittedRecords; + } + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamstatus/StatusWatermarkValveTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamstatus/StatusWatermarkValveTest.java index 4d0666b4c2..ff20a16c9b 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamstatus/StatusWatermarkValveTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/streamstatus/StatusWatermarkValveTest.java @@ -19,7 +19,10 @@ package org.apache.flink.streaming.runtime.streamstatus; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.junit.Test; @@ -49,7 +52,7 @@ public class StatusWatermarkValveTest { */ @Test public void testSingleInputIncreasingWatermarks() throws Exception { - BufferedValveOutputHandler valveOutput = new BufferedValveOutputHandler(); + StatusWatermarkOutput valveOutput = new StatusWatermarkOutput(); StatusWatermarkValve valve = new StatusWatermarkValve(1, valveOutput); valve.inputWatermark(new Watermark(0), 0); @@ -66,7 +69,7 @@ public class StatusWatermarkValveTest { */ @Test public void testSingleInputDecreasingWatermarksYieldsNoOutput() throws Exception { - BufferedValveOutputHandler valveOutput = new BufferedValveOutputHandler(); + StatusWatermarkOutput valveOutput = new StatusWatermarkOutput(); StatusWatermarkValve valve = new StatusWatermarkValve(1, valveOutput); valve.inputWatermark(new Watermark(25), 0); @@ -86,7 +89,7 @@ public class StatusWatermarkValveTest { */ @Test public void testSingleInputStreamStatusToggling() throws Exception { - BufferedValveOutputHandler valveOutput = new BufferedValveOutputHandler(); + StatusWatermarkOutput valveOutput = new StatusWatermarkOutput(); StatusWatermarkValve valve = new StatusWatermarkValve(1, valveOutput); valve.inputStreamStatus(StreamStatus.ACTIVE, 0); @@ -109,7 +112,7 @@ public class StatusWatermarkValveTest { */ @Test public void testSingleInputWatermarksIntactDuringIdleness() throws Exception { - BufferedValveOutputHandler valveOutput = new BufferedValveOutputHandler(); + StatusWatermarkOutput valveOutput = new StatusWatermarkOutput(); StatusWatermarkValve valve = new StatusWatermarkValve(1, valveOutput); valve.inputWatermark(new Watermark(25), 0); @@ -137,7 +140,7 @@ public class StatusWatermarkValveTest { */ @Test public void testMultipleInputYieldsWatermarkOnlyWhenAllChannelsReceivesWatermarks() throws Exception { - BufferedValveOutputHandler valveOutput = new BufferedValveOutputHandler(); + StatusWatermarkOutput valveOutput = new StatusWatermarkOutput(); StatusWatermarkValve valve = new StatusWatermarkValve(3, valveOutput); valve.inputWatermark(new Watermark(0), 0); @@ -156,7 +159,7 @@ public class StatusWatermarkValveTest { */ @Test public void testMultipleInputIncreasingWatermarks() throws Exception { - BufferedValveOutputHandler valveOutput = new BufferedValveOutputHandler(); + StatusWatermarkOutput valveOutput = new StatusWatermarkOutput(); StatusWatermarkValve valve = new StatusWatermarkValve(3, valveOutput); valve.inputWatermark(new Watermark(0), 0); @@ -190,7 +193,7 @@ public class StatusWatermarkValveTest { */ @Test public void testMultipleInputDecreasingWatermarksYieldsNoOutput() throws Exception { - BufferedValveOutputHandler valveOutput = new BufferedValveOutputHandler(); + StatusWatermarkOutput valveOutput = new StatusWatermarkOutput(); StatusWatermarkValve valve = new StatusWatermarkValve(3, valveOutput); valve.inputWatermark(new Watermark(25), 0); @@ -210,7 +213,7 @@ public class StatusWatermarkValveTest { */ @Test public void testMultipleInputStreamStatusToggling() throws Exception { - BufferedValveOutputHandler valveOutput = new BufferedValveOutputHandler(); + StatusWatermarkOutput valveOutput = new StatusWatermarkOutput(); StatusWatermarkValve valve = new StatusWatermarkValve(2, valveOutput); // this also implicitly verifies that all input channels start as active @@ -244,7 +247,7 @@ public class StatusWatermarkValveTest { */ @Test public void testMultipleInputWatermarkAdvancingWithPartiallyIdleChannels() throws Exception { - BufferedValveOutputHandler valveOutput = new BufferedValveOutputHandler(); + StatusWatermarkOutput valveOutput = new StatusWatermarkOutput(); StatusWatermarkValve valve = new StatusWatermarkValve(3, valveOutput); valve.inputWatermark(new Watermark(15), 0); @@ -273,7 +276,7 @@ public class StatusWatermarkValveTest { */ @Test public void testMultipleInputWatermarkAdvancingAsChannelsIndividuallyBecomeIdle() throws Exception { - BufferedValveOutputHandler valveOutput = new BufferedValveOutputHandler(); + StatusWatermarkOutput valveOutput = new StatusWatermarkOutput(); StatusWatermarkValve valve = new StatusWatermarkValve(3, valveOutput); valve.inputWatermark(new Watermark(25), 0); @@ -301,7 +304,7 @@ public class StatusWatermarkValveTest { */ @Test public void testMultipleInputFlushMaxWatermarkAndStreamStatusOnceAllInputsBecomeIdle() throws Exception { - BufferedValveOutputHandler valveOutput = new BufferedValveOutputHandler(); + StatusWatermarkOutput valveOutput = new StatusWatermarkOutput(); StatusWatermarkValve valve = new StatusWatermarkValve(3, valveOutput); // ------------------------------------------------------------------------------------------- @@ -339,7 +342,7 @@ public class StatusWatermarkValveTest { */ @Test public void testMultipleInputWatermarkRealignmentAfterResumeActive() throws Exception { - BufferedValveOutputHandler valveOutput = new BufferedValveOutputHandler(); + StatusWatermarkOutput valveOutput = new StatusWatermarkOutput(); StatusWatermarkValve valve = new StatusWatermarkValve(3, valveOutput); valve.inputWatermark(new Watermark(10), 0); @@ -380,7 +383,7 @@ public class StatusWatermarkValveTest { */ @Test public void testNoOutputWhenAllActiveChannelsAreUnaligned() throws Exception { - BufferedValveOutputHandler valveOutput = new BufferedValveOutputHandler(); + StatusWatermarkOutput valveOutput = new StatusWatermarkOutput(); StatusWatermarkValve valve = new StatusWatermarkValve(3, valveOutput); valve.inputWatermark(new Watermark(10), 0); @@ -403,22 +406,32 @@ public class StatusWatermarkValveTest { assertEquals(null, valveOutput.popLastSeenOutput()); } - private class BufferedValveOutputHandler implements StatusWatermarkValve.ValveOutputHandler { + private static class StatusWatermarkOutput implements PushingAsyncDataInput.DataOutput { + private BlockingQueue allOutputs = new LinkedBlockingQueue<>(); @Override - public void handleWatermark(Watermark watermark) { + public void emitWatermark(Watermark watermark) { allOutputs.add(watermark); } @Override - public void handleStreamStatus(StreamStatus streamStatus) { + public void emitStreamStatus(StreamStatus streamStatus) { allOutputs.add(streamStatus); } + @Override + public void emitRecord(StreamRecord record) { + throw new UnsupportedOperationException(); + } + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) { + throw new UnsupportedOperationException(); + } + public StreamElement popLastSeenOutput() { return allOutputs.poll(); } } - } -- Gitee From 8496de3c019874f223faa8ee6baddc0abbf09590 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Fri, 20 Sep 2019 15:03:18 +0200 Subject: [PATCH 013/268] [FLINK-14157][e2e] Undo jaxb rellocations for java 8 in s3. --- flink-filesystems/flink-s3-fs-hadoop/pom.xml | 23 -------------------- flink-filesystems/flink-s3-fs-presto/pom.xml | 22 ------------------- 2 files changed, 45 deletions(-) diff --git a/flink-filesystems/flink-s3-fs-hadoop/pom.xml b/flink-filesystems/flink-s3-fs-hadoop/pom.xml index 4bc57a45bf..e03c885f34 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/pom.xml +++ b/flink-filesystems/flink-s3-fs-hadoop/pom.xml @@ -106,11 +106,6 @@ under the License. com.amazon org.apache.flink.fs.s3base.shaded.com.amazon - - - javax.xml.bind - org.apache.flink.fs.s3hadoop.shaded.javax.xml.bind - org.apache.flink.runtime.util @@ -131,22 +126,4 @@ under the License. - - - - java11 - - 11 - - - - - javax.xml.bind - jaxb-api - 2.3.0 - - - - - diff --git a/flink-filesystems/flink-s3-fs-presto/pom.xml b/flink-filesystems/flink-s3-fs-presto/pom.xml index b0a93c1e0c..f589e04884 100644 --- a/flink-filesystems/flink-s3-fs-presto/pom.xml +++ b/flink-filesystems/flink-s3-fs-presto/pom.xml @@ -270,10 +270,6 @@ under the License. io.airlift org.apache.flink.fs.s3presto.shaded.io.airlift - - javax.xml.bind - org.apache.flink.fs.s3presto.shaded.javax.xml.bind - org.HdrHistogram org.apache.flink.fs.s3presto.shaded.org.HdrHistogram @@ -327,22 +323,4 @@ under the License. - - - - java11 - - 11 - - - - - javax.xml.bind - jaxb-api - 2.3.0 - - - - - -- Gitee From 250f23ddb82ad7ff71c85b6691b426e21e79fff6 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Fri, 20 Sep 2019 14:41:09 +0200 Subject: [PATCH 014/268] [FLINK-14157][e2e] Disable Streaming File Sink s3 end-to-end test for java 11. The test was disabled temporarily until a proper fix is added for FLINK-13748. The problem is that the rellocation of jaxb in only relevant for Java 11 and not Java 8. For Java 8 it actually makes Flink fail at runtime. --- tools/travis/splits/split_misc.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tools/travis/splits/split_misc.sh b/tools/travis/splits/split_misc.sh index 97f811d186..856b351bf4 100755 --- a/tools/travis/splits/split_misc.sh +++ b/tools/travis/splits/split_misc.sh @@ -54,7 +54,9 @@ run_test "Streaming SQL end-to-end test (Old planner)" "$END_TO_END_DIR/test-scr run_test "Streaming SQL end-to-end test (Blink planner)" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh blink" "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" +if [[ ${PROFILE} != *"jdk11"* ]]; then + 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" +fi 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 (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" -- Gitee From 71fa23738e7f5f582c20753a92653bc3ce1e29b8 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Tue, 17 Sep 2019 18:09:00 +0800 Subject: [PATCH 015/268] [FLINK-14015][python] Introduces PythonScalarFunctionOperator to execute Python user-defined functions - Introduces PythonScalarFunctionOperator to execute Python user-defined functions for flink planner. - Introduces BaseRowPythonScalarFunctionOperator to execute Python user-defined functions for blink planner. This closes #9707. --- .../main/java/org/apache/flink/types/Row.java | 30 ++ .../java/org/apache/flink/types/RowTest.java | 24 ++ flink-python/pom.xml | 6 + .../python/AbstractPythonFunctionRunner.java | 34 +-- .../apache/flink/python/PythonOptions.java | 48 +++ .../AbstractPythonFunctionOperator.java | 278 ++++++++++++++++++ .../AbstractPythonScalarFunctionOperator.java | 200 +++++++++++++ .../BaseRowPythonScalarFunctionOperator.java | 178 +++++++++++ .../python/PythonScalarFunctionOperator.java | 158 ++++++++++ .../AbstractPythonScalarFunctionRunner.java | 11 +- .../BaseRowPythonScalarFunctionRunner.java | 9 +- .../python/PythonScalarFunctionRunner.java | 9 +- .../typeutils}/BeamTypeUtils.java | 6 +- .../typeutils}/coders/BaseRowCoder.java | 2 +- .../coders/ReusableDataInputView.java | 2 +- .../coders/ReusableDataOutputView.java | 2 +- .../typeutils}/coders/RowCoder.java | 2 +- .../flink/python/PythonOptionsTest.java | 59 ++++ ...bstractPythonScalarFunctionRunnerTest.java | 1 + ...BaseRowPythonScalarFunctionRunnerTest.java | 6 +- .../functions/python/BeamTypeUtilsTest.java | 5 +- .../PythonScalarFunctionRunnerTest.java | 12 +- ...seRowPythonScalarFunctionOperatorTest.java | 103 +++++++ .../PassThroughPythonFunctionRunner.java | 75 +++++ .../PythonScalarFunctionOperatorTest.java | 81 +++++ .../PythonScalarFunctionOperatorTestBase.java | 223 ++++++++++++++ .../typeutils}/coders/BaseRowCoderTest.java | 2 +- .../typeutils}/coders/CoderTestBase.java | 2 +- .../typeutils}/coders/RowCoderTest.java | 2 +- .../AbstractStreamOperatorTestHarness.java | 7 + 30 files changed, 1531 insertions(+), 46 deletions(-) create mode 100644 flink-python/src/main/java/org/apache/flink/python/PythonOptions.java create mode 100644 flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractPythonScalarFunctionOperator.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperator.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperator.java rename flink-python/src/main/java/org/apache/flink/table/{functions => runtime/runners}/python/AbstractPythonScalarFunctionRunner.java (95%) rename flink-python/src/main/java/org/apache/flink/table/{functions => runtime/runners}/python/BaseRowPythonScalarFunctionRunner.java (87%) rename flink-python/src/main/java/org/apache/flink/table/{functions => runtime/runners}/python/PythonScalarFunctionRunner.java (86%) rename flink-python/src/main/java/org/apache/flink/table/{functions/python => runtime/typeutils}/BeamTypeUtils.java (98%) rename flink-python/src/main/java/org/apache/flink/table/{functions/python => runtime/typeutils}/coders/BaseRowCoder.java (98%) rename flink-python/src/main/java/org/apache/flink/table/{functions/python => runtime/typeutils}/coders/ReusableDataInputView.java (95%) rename flink-python/src/main/java/org/apache/flink/table/{functions/python => runtime/typeutils}/coders/ReusableDataOutputView.java (95%) rename flink-python/src/main/java/org/apache/flink/table/{functions/python => runtime/typeutils}/coders/RowCoder.java (98%) create mode 100644 flink-python/src/test/java/org/apache/flink/python/PythonOptionsTest.java create mode 100644 flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperatorTest.java create mode 100644 flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PassThroughPythonFunctionRunner.java create mode 100644 flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTest.java create mode 100644 flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTestBase.java rename flink-python/src/test/java/org/apache/flink/table/{functions/python => runtime/typeutils}/coders/BaseRowCoderTest.java (98%) rename flink-python/src/test/java/org/apache/flink/table/{functions/python => runtime/typeutils}/coders/CoderTestBase.java (98%) rename flink-python/src/test/java/org/apache/flink/table/{functions/python => runtime/typeutils}/coders/RowCoderTest.java (96%) diff --git a/flink-core/src/main/java/org/apache/flink/types/Row.java b/flink-core/src/main/java/org/apache/flink/types/Row.java index b8bdbf9260..aa15bf9e89 100644 --- a/flink-core/src/main/java/org/apache/flink/types/Row.java +++ b/flink-core/src/main/java/org/apache/flink/types/Row.java @@ -167,4 +167,34 @@ public class Row implements Serializable{ } return newRow; } + + /** + * Creates a new Row which fields are copied from the other rows. + * This method does not perform a deep copy. + * + * @param first The first row being copied. + * @param remainings The other rows being copied. + * @return the joined new Row + */ + public static Row join(Row first, Row... remainings) { + int newLength = first.fields.length; + for (Row remaining : remainings) { + newLength += remaining.fields.length; + } + + final Row joinedRow = new Row(newLength); + int index = 0; + + // copy the first row + System.arraycopy(first.fields, 0, joinedRow.fields, index, first.fields.length); + index += first.fields.length; + + // copy the remaining rows + for (Row remaining : remainings) { + System.arraycopy(remaining.fields, 0, joinedRow.fields, index, remaining.fields.length); + index += remaining.fields.length; + } + + return joinedRow; + } } diff --git a/flink-core/src/test/java/org/apache/flink/types/RowTest.java b/flink-core/src/test/java/org/apache/flink/types/RowTest.java index 067992ab1b..605ac9b69b 100644 --- a/flink-core/src/test/java/org/apache/flink/types/RowTest.java +++ b/flink-core/src/test/java/org/apache/flink/types/RowTest.java @@ -79,4 +79,28 @@ public class RowTest { expected.setField(2, "hello world"); assertEquals(expected, projected); } + + @Test + public void testRowJoin() { + Row row1 = new Row(2); + row1.setField(0, 1); + row1.setField(1, "hello"); + + Row row2 = new Row(2); + row2.setField(0, null); + row2.setField(1, new Tuple2<>(2, "hi")); + + Row row3 = new Row(1); + row3.setField(0, "hello world"); + + Row joinedRow = Row.join(row1, row2, row3); + + Row expected = new Row(5); + expected.setField(0, 1); + expected.setField(1, "hello"); + expected.setField(2, null); + expected.setField(3, new Tuple2<>(2, "hi")); + expected.setField(4, "hello world"); + assertEquals(expected, joinedRow); + } } diff --git a/flink-python/pom.xml b/flink-python/pom.xml index fcbdb0c4d6..ed16daa09e 100644 --- a/flink-python/pom.xml +++ b/flink-python/pom.xml @@ -74,6 +74,12 @@ under the License. ${project.version} provided + + org.apache.flink + flink-table-planner-blink_${scala.binary.version} + ${project.version} + provided + diff --git a/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java index 498e090d17..73c753544a 100644 --- a/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java @@ -49,6 +49,7 @@ import java.io.DataOutputStream; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.Random; @@ -92,9 +93,9 @@ public abstract class AbstractPythonFunctionRunner implements PythonFun private final StateRequestHandler stateRequestHandler; /** - * Temporary directory to store the retrieval token. + * Temporary directories to store the retrieval token. */ - private final String tempDir; + private final String[] tempDirs; /** * The file of the retrieval token representing the entirety of the staged artifacts. @@ -146,12 +147,12 @@ public abstract class AbstractPythonFunctionRunner implements PythonFun FnDataReceiver resultReceiver, PythonEnv pythonEnv, StateRequestHandler stateRequestHandler, - String tempDir) { + String[] tempDirs) { this.taskName = Preconditions.checkNotNull(taskName); this.resultReceiver = Preconditions.checkNotNull(resultReceiver); this.pythonEnv = Preconditions.checkNotNull(pythonEnv); this.stateRequestHandler = Preconditions.checkNotNull(stateRequestHandler); - this.tempDir = Preconditions.checkNotNull(tempDir); + this.tempDirs = Preconditions.checkNotNull(tempDirs); } @Override @@ -249,23 +250,22 @@ public abstract class AbstractPythonFunctionRunner implements PythonFun } private String createEmptyRetrievalToken() throws Exception { - // try to find a unique file name for the spilling channel + // try to find a unique file name for the retrieval token final Random rnd = new Random(); for (int attempt = 0; attempt < 10; attempt++) { - retrievalToken = new File(tempDir, randomString(rnd) + ".json"); - if (!retrievalToken.exists()) { - break; + String directory = tempDirs[rnd.nextInt(tempDirs.length)]; + retrievalToken = new File(directory, randomString(rnd) + ".json"); + if (retrievalToken.createNewFile()) { + final DataOutputStream dos = new DataOutputStream(new FileOutputStream(retrievalToken)); + dos.writeBytes("{\"manifest\": {}}"); + dos.flush(); + dos.close(); + return retrievalToken.getAbsolutePath(); } } - if (retrievalToken.exists()) { - throw new IOException( - "Could not find a unique file name in '" + tempDir + "' for retrieval token."); - } - final DataOutputStream dos = new DataOutputStream(new FileOutputStream(retrievalToken)); - dos.writeBytes("{\"manifest\": {}}"); - dos.flush(); - dos.close(); - return retrievalToken.getAbsolutePath(); + + throw new IOException( + "Could not find a unique file name in '" + Arrays.toString(tempDirs) + "' for retrieval token."); } private static String randomString(Random random) { diff --git a/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java b/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java new file mode 100644 index 0000000000..6c0ee52d3e --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.python; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** + * Configuration options for the Python API. + */ +@PublicEvolving +public class PythonOptions { + + /** + * The maximum number of elements to include in a bundle. + */ + public static final ConfigOption MAX_BUNDLE_SIZE = ConfigOptions + .key("python.fn-execution.bundle.size") + .defaultValue(1000) + .withDescription("The maximum number of elements to include in a bundle. The elements " + + "are processed asynchronously. One bundle of elements are processed before " + + "processing the next bundle of elements"); + + /** + * The maximum time to wait before finalising a bundle (in milliseconds). + */ + public static final ConfigOption MAX_BUNDLE_TIME_MILLS = ConfigOptions + .key("python.fn-execution.bundle.time") + .defaultValue(1000L) + .withDescription("The maximum time to wait before finishing a bundle (in milliseconds)."); +} diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java new file mode 100644 index 0000000000..1c869db2ca --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java @@ -0,0 +1,278 @@ +/* + * 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.operators.python; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.python.PythonFunctionRunner; +import org.apache.flink.python.PythonOptions; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +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 java.util.Map; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Base class for all stream operators to execute Python functions. + */ +@Internal +public abstract class AbstractPythonFunctionOperator + extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { + + private static final long serialVersionUID = 1L; + + /** + * The {@link PythonFunctionRunner} which is responsible for Python user-defined function execution. + */ + private transient PythonFunctionRunner pythonFunctionRunner; + + /** + * Use an AtomicBoolean because we start/stop bundles by a timer thread. + */ + private transient AtomicBoolean bundleStarted; + + /** + * Number of processed elements in the current bundle. + */ + private transient int elementCount; + + /** + * Max number of elements to include in a bundle. + */ + private transient int maxBundleSize; + + /** + * Max duration of a bundle. + */ + private transient long maxBundleTimeMills; + + /** + * Time that the last bundle was finished. + */ + private transient long lastFinishBundleTime; + + /** + * A timer that finishes the current bundle after a fixed amount of time. + */ + private transient ScheduledFuture checkFinishBundleTimer; + + /** + * Callback to be executed after the current bundle was finished. + */ + private transient Runnable bundleFinishedCallback; + + @Override + public void open() throws Exception { + try { + this.bundleStarted = new AtomicBoolean(false); + + Map jobParams = getExecutionConfig().getGlobalJobParameters().toMap(); + + this.maxBundleSize = Integer.valueOf(jobParams.getOrDefault( + PythonOptions.MAX_BUNDLE_SIZE.key(), + String.valueOf(PythonOptions.MAX_BUNDLE_SIZE.defaultValue()))); + if (this.maxBundleSize <= 0) { + this.maxBundleSize = PythonOptions.MAX_BUNDLE_SIZE.defaultValue(); + LOG.error("Invalid value for the maximum bundle size. Using default value of " + + this.maxBundleSize + '.'); + } else { + LOG.info("The maximum bundle size is configured to {}.", this.maxBundleSize); + } + + this.maxBundleTimeMills = Long.valueOf(jobParams.getOrDefault( + PythonOptions.MAX_BUNDLE_TIME_MILLS.key(), + String.valueOf(PythonOptions.MAX_BUNDLE_TIME_MILLS.defaultValue()))); + if (this.maxBundleTimeMills <= 0L) { + this.maxBundleTimeMills = PythonOptions.MAX_BUNDLE_TIME_MILLS.defaultValue(); + LOG.error("Invalid value for the maximum bundle time. Using default value of " + + this.maxBundleTimeMills + '.'); + } else { + LOG.info("The maximum bundle time is configured to {} milliseconds.", this.maxBundleTimeMills); + } + + this.pythonFunctionRunner = createPythonFunctionRunner(); + this.pythonFunctionRunner.open(); + + this.elementCount = 0; + this.lastFinishBundleTime = getProcessingTimeService().getCurrentProcessingTime(); + + // Schedule timer to check timeout of finish bundle. + long bundleCheckPeriod = Math.max(this.maxBundleTimeMills, 1); + this.checkFinishBundleTimer = + getProcessingTimeService() + .scheduleAtFixedRate( + // ProcessingTimeService callbacks are executed under the checkpointing lock + timestamp -> checkInvokeFinishBundleByTime(), bundleCheckPeriod, bundleCheckPeriod); + } finally { + super.open(); + } + } + + @Override + public void close() throws Exception { + try { + invokeFinishBundle(); + } finally { + super.close(); + } + } + + @Override + public void dispose() throws Exception { + try { + if (checkFinishBundleTimer != null) { + checkFinishBundleTimer.cancel(true); + checkFinishBundleTimer = null; + } + if (pythonFunctionRunner != null) { + pythonFunctionRunner.close(); + pythonFunctionRunner = null; + } + } finally { + super.dispose(); + } + } + + @Override + public void endInput() throws Exception { + invokeFinishBundle(); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + checkInvokeStartBundle(); + pythonFunctionRunner.processElement(element.getValue()); + checkInvokeFinishBundleByCount(); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { + try { + // Ensures that no new bundle gets started + invokeFinishBundle(); + } finally { + super.prepareSnapshotPreBarrier(checkpointId); + } + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + // Due to the asynchronous communication with the SDK harness, + // a bundle might still be in progress and not all items have + // yet been received from the SDK harness. If we just set this + // watermark as the new output watermark, we could violate the + // order of the records, i.e. pending items in the SDK harness + // could become "late" although they were "on time". + // + // We can solve this problem using one of the following options: + // + // 1) Finish the current bundle and emit this watermark as the + // new output watermark. Finishing the bundle ensures that + // all the items have been processed by the SDK harness and + // the execution results sent to the downstream operator. + // + // 2) Hold on the output watermark for as long as the current + // bundle has not been finished. We have to remember to manually + // finish the bundle in case we receive the final watermark. + // To avoid latency, we should process this watermark again as + // soon as the current bundle is finished. + // + // Approach 1) is the easiest and gives better latency, yet 2) + // gives better throughput due to the bundle not getting cut on + // every watermark. So we have implemented 2) below. + if (mark.getTimestamp() == Long.MAX_VALUE) { + invokeFinishBundle(); + super.processWatermark(mark); + } else if (!bundleStarted.get()) { + // forward the watermark immediately if the bundle is already finished. + super.processWatermark(mark); + } else { + // It is not safe to advance the output watermark yet, so add a hold on the current + // output watermark. + bundleFinishedCallback = + () -> { + try { + // at this point the bundle is finished, allow the watermark to pass + super.processWatermark(mark); + } catch (Exception e) { + throw new RuntimeException( + "Failed to process watermark after finished bundle.", e); + } + }; + } + } + + /** + * Creates the {@link PythonFunctionRunner} which is responsible for Python user-defined function execution. + */ + public abstract PythonFunctionRunner createPythonFunctionRunner(); + + /** + * Sends the execution results to the downstream operator. + */ + public abstract void emitResults(); + + /** + * Checks whether to invoke startBundle. + */ + private void checkInvokeStartBundle() throws Exception { + if (bundleStarted.compareAndSet(false, true)) { + pythonFunctionRunner.startBundle(); + } + } + + /** + * Checks whether to invoke finishBundle by elements count. Called in processElement. + */ + private void checkInvokeFinishBundleByCount() throws Exception { + elementCount++; + if (elementCount >= maxBundleSize) { + invokeFinishBundle(); + } + } + + /** + * Checks whether to invoke finishBundle by timeout. + */ + private void checkInvokeFinishBundleByTime() throws Exception { + long now = getProcessingTimeService().getCurrentProcessingTime(); + if (now - lastFinishBundleTime >= maxBundleTimeMills) { + invokeFinishBundle(); + } + } + + private void invokeFinishBundle() throws Exception { + if (bundleStarted.compareAndSet(true, false)) { + pythonFunctionRunner.finishBundle(); + + emitResults(); + elementCount = 0; + lastFinishBundleTime = getProcessingTimeService().getCurrentProcessingTime(); + // callback only after current bundle was fully finalized + if (bundleFinishedCallback != null) { + bundleFinishedCallback.run(); + bundleFinishedCallback = null; + } + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractPythonScalarFunctionOperator.java new file mode 100644 index 0000000000..4ce8a9ce7e --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractPythonScalarFunctionOperator.java @@ -0,0 +1,200 @@ +/* + * 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.operators.python; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.python.PythonFunctionRunner; +import org.apache.flink.streaming.api.operators.python.AbstractPythonFunctionOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.python.PythonFunctionInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Preconditions; + +import org.apache.beam.sdk.fn.data.FnDataReceiver; + +import java.util.Arrays; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.stream.Collectors; + +/** + * Base class for all stream operators to execute Python {@link ScalarFunction}s. It executes the Python + * {@link ScalarFunction}s in separate Python execution environment. + * + *

The inputs are assumed as the following format: + * {{{ + * +------------------+--------------+ + * | forwarded fields | extra fields | + * +------------------+--------------+ + * }}}. + * + *

The Python UDFs may take input columns directly from the input row or the execution result of Java UDFs: + * 1) The input columns from the input row can be referred from the 'forwarded fields'; + * 2) The Java UDFs will be computed and the execution results can be referred from the 'extra fields'. + * + *

The outputs will be as the following format: + * {{{ + * +------------------+-------------------------+ + * | forwarded fields | scalar function results | + * +------------------+-------------------------+ + * }}}. + * + * @param Type of the input elements. + * @param Type of the output elements. + * @param Type of the UDF input type. + * @param Type of the UDF input type. + */ +@Internal +public abstract class AbstractPythonScalarFunctionOperator + extends AbstractPythonFunctionOperator { + + private static final long serialVersionUID = 1L; + + /** + * The Python {@link ScalarFunction}s to be executed. + */ + protected final PythonFunctionInfo[] scalarFunctions; + + /** + * The input logical type. + */ + protected final RowType inputType; + + /** + * The output logical type. + */ + protected final RowType outputType; + + /** + * The offsets of udf inputs. + */ + protected final int[] udfInputOffsets; + + /** + * The number of forwarded fields in the input element. + */ + protected final int forwardedFieldCnt; + + /** + * The udf input logical type. + */ + protected transient RowType udfInputType; + + /** + * The udf output logical type. + */ + protected transient RowType udfOutputType; + + /** + * The queue holding the input elements for which the execution results have not been received. + */ + protected transient LinkedBlockingQueue forwardedInputQueue; + + /** + * The queue holding the user-defined function execution results. The execution results are in + * the same order as the input elements. + */ + protected transient LinkedBlockingQueue udfResultQueue; + + AbstractPythonScalarFunctionOperator( + PythonFunctionInfo[] scalarFunctions, + RowType inputType, + RowType outputType, + int[] udfInputOffsets, + int forwardedFieldCnt) { + this.scalarFunctions = Preconditions.checkNotNull(scalarFunctions); + this.inputType = Preconditions.checkNotNull(inputType); + this.outputType = Preconditions.checkNotNull(outputType); + this.udfInputOffsets = Preconditions.checkNotNull(udfInputOffsets); + this.forwardedFieldCnt = forwardedFieldCnt; + } + + @Override + public void open() throws Exception { + forwardedInputQueue = new LinkedBlockingQueue<>(); + udfResultQueue = new LinkedBlockingQueue<>(); + udfInputType = new RowType( + Arrays.stream(udfInputOffsets) + .mapToObj(i -> inputType.getFields().get(i)) + .collect(Collectors.toList())); + udfOutputType = new RowType(outputType.getFields().subList(forwardedFieldCnt, outputType.getFieldCount())); + super.open(); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + bufferInput(element.getValue()); + super.processElement(element); + emitResults(); + } + + @Override + public PythonFunctionRunner createPythonFunctionRunner() { + final FnDataReceiver udfResultReceiver = input -> { + // handover to queue, do not block the result receiver thread + udfResultQueue.put(input); + }; + + return new ProjectUdfInputPythonScalarFunctionRunner(createPythonFunctionRunner(udfResultReceiver)); + } + + /** + * Buffers the specified input, it will be used to construct + * the operator result together with the udf execution result. + */ + public abstract void bufferInput(IN input); + + public abstract UDFIN getUdfInput(IN element); + + public abstract PythonFunctionRunner createPythonFunctionRunner(FnDataReceiver resultReceiver); + + private class ProjectUdfInputPythonScalarFunctionRunner implements PythonFunctionRunner { + + private final PythonFunctionRunner pythonFunctionRunner; + + ProjectUdfInputPythonScalarFunctionRunner(PythonFunctionRunner pythonFunctionRunner) { + this.pythonFunctionRunner = pythonFunctionRunner; + } + + @Override + public void open() throws Exception { + pythonFunctionRunner.open(); + } + + @Override + public void close() throws Exception { + pythonFunctionRunner.close(); + } + + @Override + public void startBundle() throws Exception { + pythonFunctionRunner.startBundle(); + } + + @Override + public void finishBundle() throws Exception { + pythonFunctionRunner.finishBundle(); + } + + @Override + public void processElement(IN element) throws Exception { + pythonFunctionRunner.processElement(getUdfInput(element)); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperator.java new file mode 100644 index 0000000000..2f53808f5b --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperator.java @@ -0,0 +1,178 @@ +/* + * 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.operators.python; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.python.PythonFunctionRunner; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.dataformat.BinaryRow; +import org.apache.flink.table.dataformat.JoinedRow; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.python.PythonFunctionInfo; +import org.apache.flink.table.planner.codegen.CodeGeneratorContext; +import org.apache.flink.table.planner.codegen.ProjectionCodeGenerator; +import org.apache.flink.table.runtime.generated.GeneratedProjection; +import org.apache.flink.table.runtime.generated.Projection; +import org.apache.flink.table.runtime.runners.python.BaseRowPythonScalarFunctionRunner; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Collector; + +import org.apache.beam.sdk.fn.data.FnDataReceiver; + +/** + * The Python {@link ScalarFunction} operator for the blink planner. + */ +@Internal +public class BaseRowPythonScalarFunctionOperator + extends AbstractPythonScalarFunctionOperator { + + private static final long serialVersionUID = 1L; + + /** + * The collector used to collect records. + */ + private transient StreamRecordBaseRowWrappingCollector baseRowWrapper; + + /** + * The JoinedRow reused holding the execution result. + */ + private transient JoinedRow reuseJoinedRow; + + /** + * The Projection which projects the forwarded fields from the input row. + */ + private transient Projection forwardedFieldProjection; + + /** + * The Projection which projects the udf input fields from the input row. + */ + private transient Projection udfInputProjection; + + public BaseRowPythonScalarFunctionOperator( + PythonFunctionInfo[] scalarFunctions, + RowType inputType, + RowType outputType, + int[] udfInputOffsets, + int forwardedFieldCnt) { + super(scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFieldCnt); + } + + @Override + public void open() throws Exception { + super.open(); + baseRowWrapper = new StreamRecordBaseRowWrappingCollector(output); + reuseJoinedRow = new JoinedRow(); + + udfInputProjection = createUdfInputProjection(); + forwardedFieldProjection = createForwardedFieldProjection(); + } + + @Override + public void bufferInput(BaseRow input) { + // always copy the projection result as the generated Projection reuses the projection result + BaseRow forwardedFields = forwardedFieldProjection.apply(input).copy(); + forwardedFields.setHeader(input.getHeader()); + forwardedInputQueue.add(forwardedFields); + } + + @Override + public BaseRow getUdfInput(BaseRow element) { + return udfInputProjection.apply(element); + } + + @Override + @SuppressWarnings("ConstantConditions") + public void emitResults() { + BaseRow udfResult; + while ((udfResult = udfResultQueue.poll()) != null) { + BaseRow input = forwardedInputQueue.poll(); + reuseJoinedRow.setHeader(input.getHeader()); + baseRowWrapper.collect(reuseJoinedRow.replace(input, udfResult)); + } + } + + @Override + public PythonFunctionRunner createPythonFunctionRunner(FnDataReceiver resultReceiver) { + return new BaseRowPythonScalarFunctionRunner( + getRuntimeContext().getTaskName(), + resultReceiver, + scalarFunctions, + scalarFunctions[0].getPythonFunction().getPythonEnv(), + udfInputType, + udfOutputType, + getContainingTask().getEnvironment().getTaskManagerInfo().getTmpDirectories()); + } + + private Projection createUdfInputProjection() { + final GeneratedProjection generatedProjection = ProjectionCodeGenerator.generateProjection( + CodeGeneratorContext.apply(new TableConfig()), + "UdfInputProjection", + inputType, + udfInputType, + udfInputOffsets); + // noinspection unchecked + return generatedProjection.newInstance(Thread.currentThread().getContextClassLoader()); + } + + private Projection createForwardedFieldProjection() { + final int[] fields = new int[forwardedFieldCnt]; + for (int i = 0; i < fields.length; i++) { + fields[i] = i; + } + + final RowType forwardedFieldType = new RowType(inputType.getFields().subList(0, forwardedFieldCnt)); + final GeneratedProjection generatedProjection = ProjectionCodeGenerator.generateProjection( + CodeGeneratorContext.apply(new TableConfig()), + "ForwardedFieldProjection", + inputType, + forwardedFieldType, + fields); + // noinspection unchecked + return generatedProjection.newInstance(Thread.currentThread().getContextClassLoader()); + } + + /** + * The collector is used to convert a {@link BaseRow} to a {@link StreamRecord}. + */ + private static class StreamRecordBaseRowWrappingCollector implements Collector { + + private final Collector> out; + + /** + * For Table API & SQL jobs, the timestamp field is not used. + */ + private final StreamRecord reuseStreamRecord = new StreamRecord<>(null); + + StreamRecordBaseRowWrappingCollector(Collector> out) { + this.out = out; + } + + @Override + public void collect(BaseRow record) { + out.collect(reuseStreamRecord.replace(record)); + } + + @Override + public void close() { + out.close(); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperator.java new file mode 100644 index 0000000000..9961b16548 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperator.java @@ -0,0 +1,158 @@ +/* + * 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.operators.python; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.python.PythonFunctionRunner; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.python.PythonFunctionInfo; +import org.apache.flink.table.runtime.runners.python.PythonScalarFunctionRunner; +import org.apache.flink.table.runtime.types.CRow; +import org.apache.flink.table.runtime.types.CRowTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; + +import org.apache.beam.sdk.fn.data.FnDataReceiver; + +/** + * The Python {@link ScalarFunction} operator for the legacy planner. + */ +@Internal +public class PythonScalarFunctionOperator extends AbstractPythonScalarFunctionOperator { + + private static final long serialVersionUID = 1L; + + /** + * The collector used to collect records. + */ + private transient StreamRecordCRowWrappingCollector cRowWrapper; + + /** + * The type serializer for the forwarded fields. + */ + private transient TypeSerializer forwardedInputSerializer; + + public PythonScalarFunctionOperator( + PythonFunctionInfo[] scalarFunctions, + RowType inputType, + RowType outputType, + int[] udfInputOffsets, + int forwardedFieldCnt) { + super(scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFieldCnt); + } + + @Override + public void open() throws Exception { + super.open(); + this.cRowWrapper = new StreamRecordCRowWrappingCollector(output); + + CRowTypeInfo forwardedInputTypeInfo = new CRowTypeInfo(new RowTypeInfo( + inputType.getFields().stream() + .limit(forwardedFieldCnt) + .map(RowType.RowField::getType) + .map(TypeConversions::fromLogicalToDataType) + .map(TypeConversions::fromDataTypeToLegacyInfo) + .toArray(TypeInformation[]::new))); + forwardedInputSerializer = forwardedInputTypeInfo.createSerializer(getExecutionConfig()); + } + + @Override + public void bufferInput(CRow input) { + CRow forwardedFieldsRow = new CRow(getForwardedRow(input.row()), input.change()); + if (getExecutionConfig().isObjectReuseEnabled()) { + forwardedFieldsRow = forwardedInputSerializer.copy(forwardedFieldsRow); + } + forwardedInputQueue.add(forwardedFieldsRow); + } + + @Override + public Row getUdfInput(CRow element) { + return Row.project(element.row(), udfInputOffsets); + } + + @Override + @SuppressWarnings("ConstantConditions") + public void emitResults() { + Row udfResult; + while ((udfResult = udfResultQueue.poll()) != null) { + CRow input = forwardedInputQueue.poll(); + cRowWrapper.setChange(input.change()); + cRowWrapper.collect(Row.join(input.row(), udfResult)); + } + } + + @Override + public PythonFunctionRunner createPythonFunctionRunner(FnDataReceiver resultReceiver) { + return new PythonScalarFunctionRunner( + getRuntimeContext().getTaskName(), + resultReceiver, + scalarFunctions, + scalarFunctions[0].getPythonFunction().getPythonEnv(), + udfInputType, + udfOutputType, + getContainingTask().getEnvironment().getTaskManagerInfo().getTmpDirectories()); + } + + private Row getForwardedRow(Row input) { + Row row = new Row(forwardedFieldCnt); + for (int i = 0; i < row.getArity(); i++) { + row.setField(i, input.getField(i)); + } + return row; + } + + /** + * The collector is used to convert a {@link Row} to a {@link CRow}. + */ + private static class StreamRecordCRowWrappingCollector implements Collector { + + private final Collector> out; + private final CRow reuseCRow = new CRow(); + + /** + * For Table API & SQL jobs, the timestamp field is not used. + */ + private final StreamRecord reuseStreamRecord = new StreamRecord<>(reuseCRow); + + StreamRecordCRowWrappingCollector(Collector> out) { + this.out = out; + } + + public void setChange(boolean change) { + this.reuseCRow.change_$eq(change); + } + + @Override + public void collect(Row record) { + reuseCRow.row_$eq(record); + out.collect(reuseStreamRecord); + } + + @Override + public void close() { + out.close(); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/functions/python/AbstractPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonScalarFunctionRunner.java similarity index 95% rename from flink-python/src/main/java/org/apache/flink/table/functions/python/AbstractPythonScalarFunctionRunner.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonScalarFunctionRunner.java index 104b058505..2679156378 100644 --- a/flink-python/src/main/java/org/apache/flink/table/functions/python/AbstractPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonScalarFunctionRunner.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.functions.python; +package org.apache.flink.table.runtime.runners.python; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; @@ -24,6 +24,9 @@ import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.python.AbstractPythonFunctionRunner; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.python.PythonEnv; +import org.apache.flink.table.functions.python.PythonFunctionInfo; +import org.apache.flink.table.runtime.typeutils.BeamTypeUtils; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; @@ -78,8 +81,8 @@ public abstract class AbstractPythonScalarFunctionRunner extends Abstra PythonEnv pythonEnv, RowType inputType, RowType outputType, - String tempDir) { - super(taskName, resultReceiver, pythonEnv, StateRequestHandler.unsupported(), tempDir); + String[] tempDirs) { + super(taskName, resultReceiver, pythonEnv, StateRequestHandler.unsupported(), tempDirs); this.scalarFunctions = Preconditions.checkNotNull(scalarFunctions); this.inputType = Preconditions.checkNotNull(inputType); this.outputType = Preconditions.checkNotNull(outputType); @@ -164,7 +167,7 @@ public abstract class AbstractPythonScalarFunctionRunner extends Abstra * Gets the proto representation of the Python user-defined functions to be executed. */ @VisibleForTesting - FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { + public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { FlinkFnApi.UserDefinedFunctions.Builder builder = FlinkFnApi.UserDefinedFunctions.newBuilder(); for (PythonFunctionInfo pythonFunctionInfo : scalarFunctions) { builder.addUdfs(getUserDefinedFunctionProto(pythonFunctionInfo)); diff --git a/flink-python/src/main/java/org/apache/flink/table/functions/python/BaseRowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/BaseRowPythonScalarFunctionRunner.java similarity index 87% rename from flink-python/src/main/java/org/apache/flink/table/functions/python/BaseRowPythonScalarFunctionRunner.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/BaseRowPythonScalarFunctionRunner.java index 5f4cbce021..4bb5fc806c 100644 --- a/flink-python/src/main/java/org/apache/flink/table/functions/python/BaseRowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/BaseRowPythonScalarFunctionRunner.java @@ -16,12 +16,15 @@ * limitations under the License. */ -package org.apache.flink.table.functions.python; +package org.apache.flink.table.runtime.runners.python; import org.apache.flink.annotation.Internal; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.python.PythonEnv; +import org.apache.flink.table.functions.python.PythonFunctionInfo; +import org.apache.flink.table.runtime.typeutils.BeamTypeUtils; import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.coders.Coder; @@ -41,8 +44,8 @@ public class BaseRowPythonScalarFunctionRunner extends AbstractPythonScalarFunct PythonEnv pythonEnv, RowType inputType, RowType outputType, - String tempDir) { - super(taskName, resultReceiver, scalarFunctions, pythonEnv, inputType, outputType, tempDir); + String[] tempDirs) { + super(taskName, resultReceiver, scalarFunctions, pythonEnv, inputType, outputType, tempDirs); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/functions/python/PythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/PythonScalarFunctionRunner.java similarity index 86% rename from flink-python/src/main/java/org/apache/flink/table/functions/python/PythonScalarFunctionRunner.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/PythonScalarFunctionRunner.java index 92f2dd01ed..8edba3a00f 100644 --- a/flink-python/src/main/java/org/apache/flink/table/functions/python/PythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/PythonScalarFunctionRunner.java @@ -16,11 +16,14 @@ * limitations under the License. */ -package org.apache.flink.table.functions.python; +package org.apache.flink.table.runtime.runners.python; import org.apache.flink.annotation.Internal; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.python.PythonEnv; +import org.apache.flink.table.functions.python.PythonFunctionInfo; +import org.apache.flink.table.runtime.typeutils.BeamTypeUtils; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; @@ -41,8 +44,8 @@ public class PythonScalarFunctionRunner extends AbstractPythonScalarFunctionRunn PythonEnv pythonEnv, RowType inputType, RowType outputType, - String tempDir) { - super(taskName, resultReceiver, scalarFunctions, pythonEnv, inputType, outputType, tempDir); + String[] tempDirs) { + super(taskName, resultReceiver, scalarFunctions, pythonEnv, inputType, outputType, tempDirs); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/functions/python/BeamTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/BeamTypeUtils.java similarity index 98% rename from flink-python/src/main/java/org/apache/flink/table/functions/python/BeamTypeUtils.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/BeamTypeUtils.java index ac3e9893b9..710529cc77 100644 --- a/flink-python/src/main/java/org/apache/flink/table/functions/python/BeamTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/BeamTypeUtils.java @@ -16,12 +16,12 @@ * limitations under the License. */ -package org.apache.flink.table.functions.python; +package org.apache.flink.table.runtime.typeutils; import org.apache.flink.annotation.Internal; import org.apache.flink.fnexecution.v1.FlinkFnApi; -import org.apache.flink.table.functions.python.coders.BaseRowCoder; -import org.apache.flink.table.functions.python.coders.RowCoder; +import org.apache.flink.table.runtime.typeutils.coders.BaseRowCoder; +import org.apache.flink.table.runtime.typeutils.coders.RowCoder; import org.apache.flink.table.types.logical.AnyType; import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; diff --git a/flink-python/src/main/java/org/apache/flink/table/functions/python/coders/BaseRowCoder.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/BaseRowCoder.java similarity index 98% rename from flink-python/src/main/java/org/apache/flink/table/functions/python/coders/BaseRowCoder.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/BaseRowCoder.java index 8a07c2221f..9834c50b24 100644 --- a/flink-python/src/main/java/org/apache/flink/table/functions/python/coders/BaseRowCoder.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/BaseRowCoder.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.functions.python.coders; +package org.apache.flink.table.runtime.typeutils.coders; import org.apache.flink.annotation.Internal; import org.apache.flink.core.memory.DataOutputView; diff --git a/flink-python/src/main/java/org/apache/flink/table/functions/python/coders/ReusableDataInputView.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataInputView.java similarity index 95% rename from flink-python/src/main/java/org/apache/flink/table/functions/python/coders/ReusableDataInputView.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataInputView.java index 5beaa698f6..d71cf3bf8f 100644 --- a/flink-python/src/main/java/org/apache/flink/table/functions/python/coders/ReusableDataInputView.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataInputView.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.functions.python.coders; +package org.apache.flink.table.runtime.typeutils.coders; import org.apache.flink.annotation.Internal; import org.apache.flink.core.memory.DataInputView; diff --git a/flink-python/src/main/java/org/apache/flink/table/functions/python/coders/ReusableDataOutputView.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataOutputView.java similarity index 95% rename from flink-python/src/main/java/org/apache/flink/table/functions/python/coders/ReusableDataOutputView.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataOutputView.java index 9aabb4f8da..6128044648 100644 --- a/flink-python/src/main/java/org/apache/flink/table/functions/python/coders/ReusableDataOutputView.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataOutputView.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.functions.python.coders; +package org.apache.flink.table.runtime.typeutils.coders; import org.apache.flink.annotation.Internal; import org.apache.flink.core.memory.DataOutputView; diff --git a/flink-python/src/main/java/org/apache/flink/table/functions/python/coders/RowCoder.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/RowCoder.java similarity index 98% rename from flink-python/src/main/java/org/apache/flink/table/functions/python/coders/RowCoder.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/RowCoder.java index 5b1e8e0e0a..b72d5f117f 100644 --- a/flink-python/src/main/java/org/apache/flink/table/functions/python/coders/RowCoder.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/RowCoder.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.functions.python.coders; +package org.apache.flink.table.runtime.typeutils.coders; import org.apache.flink.annotation.Internal; import org.apache.flink.types.Row; diff --git a/flink-python/src/test/java/org/apache/flink/python/PythonOptionsTest.java b/flink-python/src/test/java/org/apache/flink/python/PythonOptionsTest.java new file mode 100644 index 0000000000..74072dc2b9 --- /dev/null +++ b/flink-python/src/test/java/org/apache/flink/python/PythonOptionsTest.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.python; + +import org.apache.flink.configuration.Configuration; + +import org.junit.Test; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +/** + * Test all configurations can be set using configuration. + */ +public class PythonOptionsTest { + + @Test + public void testBundleSize() { + final Configuration configuration = new Configuration(); + final int defaultBundleSize = configuration.getInteger(PythonOptions.MAX_BUNDLE_SIZE); + assertThat(defaultBundleSize, is(equalTo(PythonOptions.MAX_BUNDLE_SIZE.defaultValue()))); + + final int expectedBundleSize = 100; + configuration.setInteger(PythonOptions.MAX_BUNDLE_SIZE, expectedBundleSize); + + final int actualBundleSize = configuration.getInteger(PythonOptions.MAX_BUNDLE_SIZE); + assertThat(actualBundleSize, is(equalTo(expectedBundleSize))); + } + + @Test + public void testBundleTime() { + final Configuration configuration = new Configuration(); + final long defaultBundleTime = configuration.getLong(PythonOptions.MAX_BUNDLE_TIME_MILLS); + assertThat(defaultBundleTime, is(equalTo(PythonOptions.MAX_BUNDLE_TIME_MILLS.defaultValue()))); + + final long expectedBundleTime = 100; + configuration.setLong(PythonOptions.MAX_BUNDLE_TIME_MILLS, expectedBundleTime); + + final long actualBundleSize = configuration.getLong(PythonOptions.MAX_BUNDLE_TIME_MILLS); + assertThat(actualBundleSize, is(equalTo(expectedBundleTime))); + } +} diff --git a/flink-python/src/test/java/org/apache/flink/table/functions/python/AbstractPythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/functions/python/AbstractPythonScalarFunctionRunnerTest.java index b811d78f57..c53f69a06b 100644 --- a/flink-python/src/test/java/org/apache/flink/table/functions/python/AbstractPythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/functions/python/AbstractPythonScalarFunctionRunnerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.table.functions.python; +import org.apache.flink.table.runtime.runners.python.AbstractPythonScalarFunctionRunner; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.RowType; diff --git a/flink-python/src/test/java/org/apache/flink/table/functions/python/BaseRowPythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/functions/python/BaseRowPythonScalarFunctionRunnerTest.java index 4e8e655a8d..02f5b7462c 100644 --- a/flink-python/src/test/java/org/apache/flink/table/functions/python/BaseRowPythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/functions/python/BaseRowPythonScalarFunctionRunnerTest.java @@ -19,7 +19,9 @@ package org.apache.flink.table.functions.python; import org.apache.flink.table.dataformat.BaseRow; -import org.apache.flink.table.functions.python.coders.BaseRowCoder; +import org.apache.flink.table.runtime.runners.python.AbstractPythonScalarFunctionRunner; +import org.apache.flink.table.runtime.runners.python.BaseRowPythonScalarFunctionRunner; +import org.apache.flink.table.runtime.typeutils.coders.BaseRowCoder; import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.coders.Coder; @@ -123,6 +125,6 @@ public class BaseRowPythonScalarFunctionRunnerTest extends AbstractPythonScalarF pythonEnv, inputType, outputType, - System.getProperty("java.io.tmpdir")); + new String[] {System.getProperty("java.io.tmpdir")}); } } diff --git a/flink-python/src/test/java/org/apache/flink/table/functions/python/BeamTypeUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/functions/python/BeamTypeUtilsTest.java index 7acf7b3d7e..0f3f3083e2 100644 --- a/flink-python/src/test/java/org/apache/flink/table/functions/python/BeamTypeUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/functions/python/BeamTypeUtilsTest.java @@ -19,8 +19,9 @@ package org.apache.flink.table.functions.python; import org.apache.flink.fnexecution.v1.FlinkFnApi; -import org.apache.flink.table.functions.python.coders.BaseRowCoder; -import org.apache.flink.table.functions.python.coders.RowCoder; +import org.apache.flink.table.runtime.typeutils.BeamTypeUtils; +import org.apache.flink.table.runtime.typeutils.coders.BaseRowCoder; +import org.apache.flink.table.runtime.typeutils.coders.RowCoder; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.RowType; diff --git a/flink-python/src/test/java/org/apache/flink/table/functions/python/PythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/functions/python/PythonScalarFunctionRunnerTest.java index c3b63bbb6e..6a9398a202 100644 --- a/flink-python/src/test/java/org/apache/flink/table/functions/python/PythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/functions/python/PythonScalarFunctionRunnerTest.java @@ -19,7 +19,9 @@ package org.apache.flink.table.functions.python; import org.apache.flink.fnexecution.v1.FlinkFnApi; -import org.apache.flink.table.functions.python.coders.RowCoder; +import org.apache.flink.table.runtime.runners.python.AbstractPythonScalarFunctionRunner; +import org.apache.flink.table.runtime.runners.python.PythonScalarFunctionRunner; +import org.apache.flink.table.runtime.typeutils.coders.RowCoder; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; @@ -251,7 +253,7 @@ public class PythonScalarFunctionRunnerTest extends AbstractPythonScalarFunction pythonEnv, inputType, outputType, - System.getProperty("java.io.tmpdir")); + new String[] {System.getProperty("java.io.tmpdir")}); } private AbstractPythonScalarFunctionRunner createUDFRunner( @@ -274,7 +276,7 @@ public class PythonScalarFunctionRunnerTest extends AbstractPythonScalarFunction rowType, rowType, jobBundleFactory, - System.getProperty("java.io.tmpdir")); + new String[] {System.getProperty("java.io.tmpdir")}); } private static class PythonScalarFunctionRunnerTestHarness extends PythonScalarFunctionRunner { @@ -288,8 +290,8 @@ public class PythonScalarFunctionRunnerTest extends AbstractPythonScalarFunction PythonEnv pythonEnv, RowType inputType, RowType outputType, JobBundleFactory jobBundleFactory, - String tempDir) { - super(taskName, resultReceiver, scalarFunctions, pythonEnv, inputType, outputType, tempDir); + String[] tempDirs) { + super(taskName, resultReceiver, scalarFunctions, pythonEnv, inputType, outputType, tempDirs); this.jobBundleFactory = jobBundleFactory; } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperatorTest.java new file mode 100644 index 0000000000..35288cbfa7 --- /dev/null +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperatorTest.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.operators.python; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.python.PythonFunctionRunner; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.dataformat.util.BaseRowUtil; +import org.apache.flink.table.functions.python.PythonFunctionInfo; +import org.apache.flink.table.runtime.util.BaseRowHarnessAssertor; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.beam.sdk.fn.data.FnDataReceiver; + +import java.util.Collection; + +import static org.apache.flink.table.runtime.util.StreamRecordUtils.baserow; +import static org.apache.flink.table.runtime.util.StreamRecordUtils.binaryrow; + +/** + * Tests for {@link BaseRowPythonScalarFunctionOperator}. + */ +public class BaseRowPythonScalarFunctionOperatorTest + extends PythonScalarFunctionOperatorTestBase { + + private final BaseRowHarnessAssertor assertor = new BaseRowHarnessAssertor(new TypeInformation[]{ + Types.STRING, + Types.STRING, + Types.LONG + }); + + @Override + public AbstractPythonScalarFunctionOperator getTestOperator( + PythonFunctionInfo[] scalarFunctions, + RowType inputType, + RowType outputType, + int[] udfInputOffsets, + int forwardedFieldCnt) { + return new PassThroughPythonScalarFunctionOperator( + scalarFunctions, + inputType, + outputType, + udfInputOffsets, + forwardedFieldCnt + ); + } + + @Override + public BaseRow newRow(boolean accumulateMsg, Object... fields) { + if (accumulateMsg) { + return baserow(fields); + } else { + return BaseRowUtil.setRetract(baserow(fields)); + } + } + + @Override + public void assertOutputEquals(String message, Collection expected, Collection actual) { + assertor.assertOutputEquals(message, expected, actual); + } + + private static class PassThroughPythonScalarFunctionOperator extends BaseRowPythonScalarFunctionOperator { + + PassThroughPythonScalarFunctionOperator( + PythonFunctionInfo[] scalarFunctions, + RowType inputType, + RowType outputType, + int[] udfInputOffsets, + int forwardedFieldCnt) { + super(scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFieldCnt); + } + + @Override + public PythonFunctionRunner createPythonFunctionRunner( + FnDataReceiver resultReceiver) { + return new PassThroughPythonFunctionRunner(resultReceiver) { + @Override + public BaseRow copy(BaseRow element) { + BaseRow row = binaryrow(element.getLong(0)); + row.setHeader(element.getHeader()); + return row; + } + }; + } + } +} diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PassThroughPythonFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PassThroughPythonFunctionRunner.java new file mode 100644 index 0000000000..9660ab0ffe --- /dev/null +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PassThroughPythonFunctionRunner.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.table.runtime.operators.python; + +import org.apache.flink.python.PythonFunctionRunner; +import org.apache.flink.util.Preconditions; + +import org.apache.beam.sdk.fn.data.FnDataReceiver; + +import java.util.ArrayList; +import java.util.List; + +/** + * A {@link PythonFunctionRunner} that just emit each input element. + * + * @param Type of the input elements. + */ +public abstract class PassThroughPythonFunctionRunner implements PythonFunctionRunner { + + private boolean bundleStarted; + private final List bufferedElements; + private final FnDataReceiver resultReceiver; + + PassThroughPythonFunctionRunner(FnDataReceiver resultReceiver) { + this.resultReceiver = Preconditions.checkNotNull(resultReceiver); + bundleStarted = false; + bufferedElements = new ArrayList<>(); + } + + @Override + public void open() {} + + @Override + public void close() {} + + @Override + public void startBundle() { + Preconditions.checkState(!bundleStarted); + bundleStarted = true; + } + + @Override + public void finishBundle() throws Exception { + Preconditions.checkState(bundleStarted); + bundleStarted = false; + + for (IN element : bufferedElements) { + resultReceiver.accept(element); + } + bufferedElements.clear(); + } + + @Override + public void processElement(IN element) { + bufferedElements.add(copy(element)); + } + + public abstract IN copy(IN element); +} diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTest.java new file mode 100644 index 0000000000..d9a854e955 --- /dev/null +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTest.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.table.runtime.operators.python; + +import org.apache.flink.python.PythonFunctionRunner; +import org.apache.flink.streaming.util.TestHarnessUtil; +import org.apache.flink.table.functions.python.PythonFunctionInfo; +import org.apache.flink.table.runtime.types.CRow; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; + +import org.apache.beam.sdk.fn.data.FnDataReceiver; + +import java.util.Collection; +import java.util.Queue; + +/** + * Tests for {@link PythonScalarFunctionOperator}. + */ +public class PythonScalarFunctionOperatorTest extends PythonScalarFunctionOperatorTestBase { + + @Override + public AbstractPythonScalarFunctionOperator getTestOperator( + PythonFunctionInfo[] scalarFunctions, + RowType inputType, + RowType outputType, + int[] udfInputOffsets, + int forwardedFieldCnt) { + return new PassThroughPythonScalarFunctionOperator( + scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFieldCnt); + } + + @Override + public CRow newRow(boolean accumulateMsg, Object... fields) { + return new CRow(Row.of(fields), accumulateMsg); + } + + @Override + public void assertOutputEquals(String message, Collection expected, Collection actual) { + TestHarnessUtil.assertOutputEquals(message, (Queue) expected, (Queue) actual); + } + + private static class PassThroughPythonScalarFunctionOperator extends PythonScalarFunctionOperator { + + PassThroughPythonScalarFunctionOperator( + PythonFunctionInfo[] scalarFunctions, + RowType inputType, + RowType outputType, + int[] udfInputOffsets, + int forwardedFieldCnt) { + super(scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFieldCnt); + } + + @Override + public PythonFunctionRunner createPythonFunctionRunner( + FnDataReceiver resultReceiver) { + return new PassThroughPythonFunctionRunner(resultReceiver) { + @Override + public Row copy(Row element) { + return Row.copy(element); + } + }; + } + } +} diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTestBase.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTestBase.java new file mode 100644 index 0000000000..a4c52a8375 --- /dev/null +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTestBase.java @@ -0,0 +1,223 @@ +/* + * 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.operators.python; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.python.PythonOptions; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.functions.python.AbstractPythonScalarFunctionRunnerTest; +import org.apache.flink.table.functions.python.PythonFunctionInfo; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collection; +import java.util.concurrent.ConcurrentLinkedQueue; + +/** + * Base class for Python scalar function operator test. These test that: + * + *
    + *
  • Retraction flag is correctly forwarded to the downstream
  • + *
  • FinishBundle is called when checkpoint is encountered
  • + *
  • Watermarks are buffered and only sent to downstream when finishedBundle is triggered
  • + *
+ * + * @param Type of the input elements. + * @param Type of the output elements. + * @param Type of the UDF input type. + * @param Type of the UDF input type. + */ +public abstract class PythonScalarFunctionOperatorTestBase { + + @Test + public void testRetractionFieldKept() throws Exception { + OneInputStreamOperatorTestHarness testHarness = getTestHarness(); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(newRow(true, "c1", "c2", 0L), initialTime + 1)); + testHarness.processElement(new StreamRecord<>(newRow(false, "c3", "c4", 1L), initialTime + 2)); + testHarness.processElement(new StreamRecord<>(newRow(false, "c5", "c6", 2L), initialTime + 3)); + testHarness.close(); + + expectedOutput.add(new StreamRecord<>(newRow(true, "c1", "c2", 0L))); + expectedOutput.add(new StreamRecord<>(newRow(false, "c3", "c4", 1L))); + expectedOutput.add(new StreamRecord<>(newRow(false, "c5", "c6", 2L))); + + assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } + + @Test + public void testFinishBundleTriggeredOnCheckpoint() throws Exception { + OneInputStreamOperatorTestHarness testHarness = getTestHarness(); + + Configuration conf = new Configuration(); + conf.setInteger(PythonOptions.MAX_BUNDLE_SIZE, 10); + testHarness.getExecutionConfig().setGlobalJobParameters(conf); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(newRow(true, "c1", "c2", 0L), initialTime + 1)); + + // checkpoint trigger finishBundle + testHarness.prepareSnapshotPreBarrier(0L); + + expectedOutput.add(new StreamRecord<>(newRow(true, "c1", "c2", 0L))); + + assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.close(); + } + + @Test + public void testFinishBundleTriggeredByCount() throws Exception { + OneInputStreamOperatorTestHarness testHarness = getTestHarness(); + Configuration conf = new Configuration(); + conf.setInteger(PythonOptions.MAX_BUNDLE_SIZE, 2); + testHarness.getExecutionConfig().setGlobalJobParameters(conf); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(newRow(true, "c1", "c2", 0L), initialTime + 1)); + assertOutputEquals("FinishBundle should not be triggered.", expectedOutput, testHarness.getOutput()); + + testHarness.processElement(new StreamRecord<>(newRow(true, "c1", "c2", 1L), initialTime + 2)); + expectedOutput.add(new StreamRecord<>(newRow(true, "c1", "c2", 0L))); + expectedOutput.add(new StreamRecord<>(newRow(true, "c1", "c2", 1L))); + + assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.close(); + } + + @Test + public void testFinishBundleTriggeredByTime() throws Exception { + OneInputStreamOperatorTestHarness testHarness = getTestHarness(); + Configuration conf = new Configuration(); + conf.setInteger(PythonOptions.MAX_BUNDLE_SIZE, 10); + conf.setLong(PythonOptions.MAX_BUNDLE_TIME_MILLS, 1000L); + testHarness.getExecutionConfig().setGlobalJobParameters(conf); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(newRow(true, "c1", "c2", 0L), initialTime + 1)); + assertOutputEquals("FinishBundle should not be triggered.", expectedOutput, testHarness.getOutput()); + + testHarness.setProcessingTime(1000L); + expectedOutput.add(new StreamRecord<>(newRow(true, "c1", "c2", 0L))); + assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.close(); + } + + @Test + public void testFinishBundleTriggeredByClose() throws Exception { + OneInputStreamOperatorTestHarness testHarness = getTestHarness(); + Configuration conf = new Configuration(); + conf.setInteger(PythonOptions.MAX_BUNDLE_SIZE, 10); + testHarness.getExecutionConfig().setGlobalJobParameters(conf); + + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(newRow(true, "c1", "c2", 0L), initialTime + 1)); + assertOutputEquals("FinishBundle should not be triggered.", expectedOutput, testHarness.getOutput()); + + testHarness.close(); + expectedOutput.add(new StreamRecord<>(newRow(true, "c1", "c2", 0L))); + assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } + + @Test + public void testWatermarkProcessedOnFinishBundle() throws Exception { + OneInputStreamOperatorTestHarness testHarness = getTestHarness(); + Configuration conf = new Configuration(); + conf.setInteger(PythonOptions.MAX_BUNDLE_SIZE, 10); + testHarness.getExecutionConfig().setGlobalJobParameters(conf); + long initialTime = 0L; + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(newRow(true, "c1", "c2", 0L), initialTime + 1)); + testHarness.processWatermark(initialTime + 2); + assertOutputEquals("Watermark has been processed", expectedOutput, testHarness.getOutput()); + + // checkpoint trigger finishBundle + testHarness.prepareSnapshotPreBarrier(0L); + + expectedOutput.add(new StreamRecord<>(newRow(true, "c1", "c2", 0L))); + expectedOutput.add(new Watermark(initialTime + 2)); + + assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.close(); + } + + private OneInputStreamOperatorTestHarness getTestHarness() throws Exception { + RowType dataType = new RowType(Arrays.asList( + new RowType.RowField("f1", new VarCharType()), + new RowType.RowField("f2", new VarCharType()), + new RowType.RowField("f3", new BigIntType()))); + AbstractPythonScalarFunctionOperator operator = getTestOperator( + new PythonFunctionInfo[] { + new PythonFunctionInfo( + AbstractPythonScalarFunctionRunnerTest.DummyPythonFunction.INSTANCE, + new Integer[]{0}) + }, + dataType, + dataType, + new int[]{2}, + 2 + ); + + return new OneInputStreamOperatorTestHarness<>(operator); + } + + public abstract AbstractPythonScalarFunctionOperator getTestOperator( + PythonFunctionInfo[] scalarFunctions, + RowType inputType, + RowType outputType, + int[] udfInputOffsets, + int forwardedFieldCnt); + + public abstract IN newRow(boolean accumulateMsg, Object... fields); + + public abstract void assertOutputEquals(String message, Collection expected, Collection actual); +} diff --git a/flink-python/src/test/java/org/apache/flink/table/functions/python/coders/BaseRowCoderTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/BaseRowCoderTest.java similarity index 98% rename from flink-python/src/test/java/org/apache/flink/table/functions/python/coders/BaseRowCoderTest.java rename to flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/BaseRowCoderTest.java index 57eb79b086..7b5e281a3a 100644 --- a/flink-python/src/test/java/org/apache/flink/table/functions/python/coders/BaseRowCoderTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/BaseRowCoderTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.functions.python.coders; +package org.apache.flink.table.runtime.typeutils.coders; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.table.dataformat.BaseRow; diff --git a/flink-python/src/test/java/org/apache/flink/table/functions/python/coders/CoderTestBase.java b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/CoderTestBase.java similarity index 98% rename from flink-python/src/test/java/org/apache/flink/table/functions/python/coders/CoderTestBase.java rename to flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/CoderTestBase.java index 642a5447bb..308fe5dd74 100644 --- a/flink-python/src/test/java/org/apache/flink/table/functions/python/coders/CoderTestBase.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/CoderTestBase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.functions.python.coders; +package org.apache.flink.table.runtime.typeutils.coders; import org.apache.flink.testutils.CustomEqualityMatcher; import org.apache.flink.testutils.DeeplyEqualsChecker; diff --git a/flink-python/src/test/java/org/apache/flink/table/functions/python/coders/RowCoderTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/RowCoderTest.java similarity index 96% rename from flink-python/src/test/java/org/apache/flink/table/functions/python/coders/RowCoderTest.java rename to flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/RowCoderTest.java index 93b1c003e0..e8b64830b6 100644 --- a/flink-python/src/test/java/org/apache/flink/table/functions/python/coders/RowCoderTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/RowCoderTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.functions.python.coders; +package org.apache.flink.table.runtime.typeutils.coders; import org.apache.flink.types.Row; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java index c0339510f1..8c7713a913 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java @@ -544,6 +544,13 @@ public class AbstractStreamOperatorTestHarness implements AutoCloseable { operator.open(); } + /** + * Calls {@link StreamOperator#prepareSnapshotPreBarrier(long)}. + */ + public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { + operator.prepareSnapshotPreBarrier(checkpointId); + } + /** * Calls {@link StreamOperator#snapshotState(long, long, CheckpointOptions, org.apache.flink.runtime.state.CheckpointStreamFactory)}. */ -- Gitee From f373628a0e68b79ff2437ea8c8114ed8d3114091 Mon Sep 17 00:00:00 2001 From: tison Date: Mon, 23 Sep 2019 21:19:02 +0800 Subject: [PATCH 016/268] [FLINK-14113][client] Remove class JobWithJars This closes #9726. --- .../JobWithJars.java => ClientUtils.java} | 85 +------------------ .../apache/flink/client/RemoteExecutor.java | 37 ++++---- .../flink/client/program/ClusterClient.java | 58 ++++--------- .../client/program/ContextEnvironment.java | 9 +- .../flink/client/program/PackagedProgram.java | 5 +- .../flink/client/program/ClientTest.java | 18 ++-- .../avro/AvroExternalJarProgramITCase.java | 4 +- .../environment/RemoteStreamEnvironment.java | 6 +- .../client/gateway/local/LocalExecutor.java | 6 +- 9 files changed, 71 insertions(+), 157 deletions(-) rename flink-clients/src/main/java/org/apache/flink/client/{program/JobWithJars.java => ClientUtils.java} (50%) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/JobWithJars.java b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java similarity index 50% rename from flink-clients/src/main/java/org/apache/flink/client/program/JobWithJars.java rename to flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java index 6e1bf996aa..ee03705787 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/JobWithJars.java +++ b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java @@ -16,99 +16,22 @@ * limitations under the License. */ -package org.apache.flink.client.program; +package org.apache.flink.client; -import org.apache.flink.api.common.Plan; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import java.io.File; import java.io.IOException; import java.net.URISyntaxException; import java.net.URL; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.jar.JarFile; /** - * A JobWithJars is a Flink dataflow plan, together with a bunch of JAR files that contain - * the classes of the functions and libraries necessary for the execution. + * Utility functions for Flink client. */ -public class JobWithJars { - - private Plan plan; - - private List jarFiles; - - /** - * classpaths that are needed during user code execution. - */ - private List classpaths; - - private ClassLoader userCodeClassLoader; - - public JobWithJars(Plan plan, List jarFiles, List classpaths) throws IOException { - this.plan = plan; - this.jarFiles = new ArrayList(jarFiles.size()); - this.classpaths = new ArrayList(classpaths.size()); - - for (URL jarFile: jarFiles) { - checkJarFile(jarFile); - this.jarFiles.add(jarFile); - } - - for (URL path: classpaths) { - this.classpaths.add(path); - } - } - - public JobWithJars(Plan plan, URL jarFile) throws IOException { - this.plan = plan; - - checkJarFile(jarFile); - this.jarFiles = Collections.singletonList(jarFile); - this.classpaths = Collections.emptyList(); - } - - JobWithJars(Plan plan, List jarFiles, List classpaths, ClassLoader userCodeClassLoader) { - this.plan = plan; - this.jarFiles = jarFiles; - this.classpaths = classpaths; - this.userCodeClassLoader = userCodeClassLoader; - } - - /** - * Returns the plan. - */ - public Plan getPlan() { - return this.plan; - } - - /** - * Returns list of jar files that need to be submitted with the plan. - */ - public List getJarFiles() { - return this.jarFiles; - } - - /** - * Returns list of classpaths that need to be submitted with the plan. - */ - public List getClasspaths() { - return classpaths; - } - - /** - * Gets the {@link java.lang.ClassLoader} that must be used to load user code classes. - * - * @return The user code ClassLoader. - */ - public ClassLoader getUserCodeClassLoader() { - if (this.userCodeClassLoader == null) { - this.userCodeClassLoader = buildUserCodeClassLoader(jarFiles, classpaths, getClass().getClassLoader()); - } - return this.userCodeClassLoader; - } +public enum ClientUtils { + ; public static void checkJarFile(URL jar) throws IOException { File jarFile; diff --git a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java index 37f5eff772..4d0dfd9c7d 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java @@ -22,11 +22,11 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.PlanExecutor; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.JobWithJars; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.RestOptions; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import java.net.InetSocketAddress; import java.net.URL; @@ -56,17 +56,23 @@ public class RemoteExecutor extends PlanExecutor { private int defaultParallelism = 1; public RemoteExecutor(String hostname, int port) { - this(hostname, port, new Configuration(), Collections.emptyList(), - Collections.emptyList()); + this(hostname, port, new Configuration(), Collections.emptyList(), Collections.emptyList()); } - public RemoteExecutor(String hostname, int port, Configuration clientConfiguration, - List jarFiles, List globalClasspaths) { + public RemoteExecutor( + String hostname, + int port, + Configuration clientConfiguration, + List jarFiles, + List globalClasspaths) { this(new InetSocketAddress(hostname, port), clientConfiguration, jarFiles, globalClasspaths); } - public RemoteExecutor(InetSocketAddress inet, Configuration clientConfiguration, - List jarFiles, List globalClasspaths) { + public RemoteExecutor( + InetSocketAddress inet, + Configuration clientConfiguration, + List jarFiles, + List globalClasspaths) { this.clientConfiguration = clientConfiguration; this.jarFiles = jarFiles; this.globalClasspaths = globalClasspaths; @@ -111,15 +117,16 @@ public class RemoteExecutor extends PlanExecutor { public JobExecutionResult executePlan(Plan plan) throws Exception { checkNotNull(plan); - JobWithJars p = new JobWithJars(plan, this.jarFiles, this.globalClasspaths); - return executePlanWithJars(p); - } - - private JobExecutionResult executePlanWithJars(JobWithJars program) throws Exception { - checkNotNull(program); + try (ClusterClient client = new RestClusterClient<>(clientConfiguration, "RemoteExecutor")) { + ClassLoader classLoader = ClientUtils.buildUserCodeClassLoader(jarFiles, globalClasspaths, getClass().getClassLoader()); - try (ClusterClient client = new RestClusterClient<>(clientConfiguration, "RemoteExecutor")) { - return client.run(program, defaultParallelism).getJobExecutionResult(); + return client.run( + plan, + jarFiles, + globalClasspaths, + classLoader, + defaultParallelism, + SavepointRestoreSettings.none()).getJobExecutionResult(); } } } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index 0ec54ccb8d..f90435332e 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -199,38 +199,24 @@ public abstract class ClusterClient implements AutoCloseable { } } - public JobSubmissionResult run(JobWithJars program, int parallelism) throws ProgramInvocationException { - return run(program, parallelism, SavepointRestoreSettings.none()); + public JobSubmissionResult run( + Plan plan, + List libraries, + List classpaths, + ClassLoader classLoader, + int parallelism, + SavepointRestoreSettings savepointSettings) throws CompilerException, ProgramInvocationException { + + OptimizedPlan optPlan = getOptimizedPlan(compiler, plan, parallelism); + return run(optPlan, libraries, classpaths, classLoader, savepointSettings); } - /** - * Runs a program on the Flink cluster to which this client is connected. The call blocks until the - * execution is complete, and returns afterwards. - * - * @param jobWithJars The program to be executed. - * @param parallelism The default parallelism to use when running the program. The default parallelism is used - * when the program does not set a parallelism by itself. - * - * @throws CompilerException Thrown, if the compiler encounters an illegal situation. - * @throws ProgramInvocationException Thrown, if the program could not be instantiated from its jar file, - * or if the submission failed. That might be either due to an I/O problem, - * i.e. the job-manager is unreachable, or due to the fact that the - * parallel execution failed. - */ - public JobSubmissionResult run(JobWithJars jobWithJars, int parallelism, SavepointRestoreSettings savepointSettings) - throws CompilerException, ProgramInvocationException { - ClassLoader classLoader = jobWithJars.getUserCodeClassLoader(); - if (classLoader == null) { - throw new IllegalArgumentException("The given JobWithJars does not provide a usercode class loader."); - } - - OptimizedPlan optPlan = getOptimizedPlan(compiler, jobWithJars, parallelism); - return run(optPlan, jobWithJars.getJarFiles(), jobWithJars.getClasspaths(), classLoader, savepointSettings); - } - - public JobSubmissionResult run(FlinkPlan compiledPlan, - List libraries, List classpaths, ClassLoader classLoader, SavepointRestoreSettings savepointSettings) - throws ProgramInvocationException { + public JobSubmissionResult run( + FlinkPlan compiledPlan, + List libraries, + List classpaths, + ClassLoader classLoader, + SavepointRestoreSettings savepointSettings) throws ProgramInvocationException { JobGraph job = getJobGraph(flinkConfig, compiledPlan, libraries, classpaths, savepointSettings); return submitJob(job, classLoader); } @@ -317,18 +303,6 @@ public abstract class ClusterClient implements AutoCloseable { // Internal translation methods // ------------------------------------------------------------------------ - /** - * Creates the optimized plan for a given program, using this client's compiler. - * - * @param prog The program to be compiled. - * @return The compiled and optimized plan, as returned by the compiler. - * @throws CompilerException Thrown, if the compiler encounters an illegal situation. - */ - private static OptimizedPlan getOptimizedPlan(Optimizer compiler, JobWithJars prog, int parallelism) - throws CompilerException, ProgramInvocationException { - return getOptimizedPlan(compiler, prog.getPlan(), parallelism); - } - public static JobGraph getJobGraph(Configuration flinkConfig, PackagedProgram prog, FlinkPlan optPlan, SavepointRestoreSettings savepointSettings) throws ProgramInvocationException { return getJobGraph(flinkConfig, optPlan, prog.getAllLibraries(), prog.getClasspaths(), savepointSettings); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java index 3fdae5e58c..4d46595512 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java @@ -65,8 +65,13 @@ public class ContextEnvironment extends ExecutionEnvironment { verifyExecuteIsCalledOnceWhenInDetachedMode(); final Plan plan = createProgramPlan(jobName); - final JobWithJars job = new JobWithJars(plan, jarFilesToAttach, classpathsToAttach, userCodeClassLoader); - final JobSubmissionResult jobSubmissionResult = client.run(job, getParallelism(), savepointSettings); + final JobSubmissionResult jobSubmissionResult = client.run( + plan, + jarFilesToAttach, + classpathsToAttach, + userCodeClassLoader, + getParallelism(), + savepointSettings); lastJobExecutionResult = jobSubmissionResult.getJobExecutionResult(); return lastJobExecutionResult; diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java index ebc840e588..2f765b14ed 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java @@ -19,6 +19,7 @@ package org.apache.flink.client.program; import org.apache.flink.api.common.ProgramDescription; +import org.apache.flink.client.ClientUtils; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.util.InstantiationUtil; @@ -198,7 +199,7 @@ public class PackagedProgram { // now that we have an entry point, we can extract the nested jar files (if any) this.extractedTempLibraries = jarFileUrl == null ? Collections.emptyList() : extractContainedLibraries(jarFileUrl); this.classpaths = classpaths; - this.userCodeClassLoader = JobWithJars.buildUserCodeClassLoader(getAllLibraries(), classpaths, getClass().getClassLoader()); + this.userCodeClassLoader = ClientUtils.buildUserCodeClassLoader(getAllLibraries(), classpaths, getClass().getClassLoader()); // load the entry point class this.mainClass = loadMainClass(entryPointClassName, userCodeClassLoader); @@ -623,7 +624,7 @@ public class PackagedProgram { private static void checkJarFile(URL jarfile) throws ProgramInvocationException { try { - JobWithJars.checkJarFile(jarfile); + ClientUtils.checkJarFile(jarfile); } catch (IOException e) { throw new ProgramInvocationException(e.getMessage(), e); diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java index 6ceec7f808..3b54b8efd5 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java @@ -36,6 +36,7 @@ import org.apache.flink.optimizer.Optimizer; import org.apache.flink.optimizer.costs.DefaultCostEstimator; import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.testutils.MiniClusterResource; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.util.NetUtils; @@ -47,7 +48,6 @@ import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.net.URL; import java.util.Collections; import static org.junit.Assert.assertEquals; @@ -65,7 +65,7 @@ public class ClientTest extends TestLogger { public static final MiniClusterResource MINI_CLUSTER_RESOURCE = new MiniClusterResource(new MiniClusterResourceConfiguration.Builder().build()); - private JobWithJars jobWithJars; + private Plan plan; private Configuration config; @@ -77,10 +77,8 @@ public class ClientTest extends TestLogger { public void setUp() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(); - env.generateSequence(1, 1000).output(new DiscardingOutputFormat()); - - Plan plan = env.createProgramPlan(); - jobWithJars = new JobWithJars(plan, Collections.emptyList(), Collections.emptyList()); + env.generateSequence(1, 1000).output(new DiscardingOutputFormat<>()); + plan = env.createProgramPlan(); final int freePort = NetUtils.getAvailablePort(); config = new Configuration(); @@ -155,7 +153,13 @@ public class ClientTest extends TestLogger { public void shouldSubmitToJobClient() throws Exception { final ClusterClient clusterClient = new MiniClusterClient(new Configuration(), MINI_CLUSTER_RESOURCE.getMiniCluster()); clusterClient.setDetached(true); - JobSubmissionResult result = clusterClient.run(jobWithJars, 1); + JobSubmissionResult result = clusterClient.run( + plan, + Collections.emptyList(), + Collections.emptyList(), + getClass().getClassLoader(), + 1, + SavepointRestoreSettings.none()); assertNotNull(result); } diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroExternalJarProgramITCase.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroExternalJarProgramITCase.java index 0ad5d329a7..fa79a26e23 100644 --- a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroExternalJarProgramITCase.java +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroExternalJarProgramITCase.java @@ -18,7 +18,7 @@ package org.apache.flink.formats.avro; -import org.apache.flink.client.program.JobWithJars; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.core.fs.Path; import org.apache.flink.formats.avro.testjar.AvroExternalJarProgram; @@ -68,7 +68,7 @@ public class AvroExternalJarProgramITCase extends TestLogger { String jarFile = JAR_FILE; try { - JobWithJars.checkJarFile(new File(jarFile).getAbsoluteFile().toURI().toURL()); + ClientUtils.checkJarFile(new File(jarFile).getAbsoluteFile().toURI().toURL()); } catch (IOException e) { jarFile = "target/".concat(jarFile); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java index 0ba5bd6059..13bde9662e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java @@ -23,8 +23,8 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.JobWithJars; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; @@ -191,7 +191,7 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment { try { URL jarFileUrl = new File(jarFile).getAbsoluteFile().toURI().toURL(); this.jarFiles.add(jarFileUrl); - JobWithJars.checkJarFile(jarFileUrl); + ClientUtils.checkJarFile(jarFileUrl); } catch (MalformedURLException e) { throw new IllegalArgumentException("JAR file path is invalid '" + jarFile + "'", e); } catch (IOException e) { @@ -255,7 +255,7 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment { LOG.info("Running remotely at {}:{}", host, port); } - ClassLoader userCodeClassLoader = JobWithJars.buildUserCodeClassLoader(jarFiles, globalClasspaths, envClassLoader); + ClassLoader userCodeClassLoader = ClientUtils.buildUserCodeClassLoader(jarFiles, globalClasspaths, envClassLoader); Configuration configuration = new Configuration(); configuration.addAll(clientConfiguration); diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java index 16f1161c6e..aff7e7daff 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java @@ -20,12 +20,12 @@ package org.apache.flink.table.client.gateway.local; import org.apache.flink.api.common.JobID; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.cli.CliFrontend; import org.apache.flink.client.cli.CliFrontendParser; import org.apache.flink.client.cli.CustomCommandLine; import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.JobWithJars; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.core.fs.FileSystem; @@ -581,7 +581,7 @@ public class LocalExecutor implements Executor { try { // find jar files for (URL url : jars) { - JobWithJars.checkJarFile(url); + ClientUtils.checkJarFile(url); dependencies.add(url); } @@ -601,7 +601,7 @@ public class LocalExecutor implements Executor { // only consider jars if (f.isFile() && f.getAbsolutePath().toLowerCase().endsWith(".jar")) { final URL url = f.toURI().toURL(); - JobWithJars.checkJarFile(url); + ClientUtils.checkJarFile(url); dependencies.add(url); } } -- Gitee From 627e7cb1f265e8cf8e953c17ee492ad957539f2f Mon Sep 17 00:00:00 2001 From: unknown Date: Tue, 17 Sep 2019 17:19:42 +0800 Subject: [PATCH 017/268] [FLINK-14094][metrics] Avoid duplicate metrics registration in TaskMetricGroup.getOrAddOperator This closes #9697. --- .../metrics/groups/TaskMetricGroup.java | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskMetricGroup.java index 39d98d82fc..b5f389be66 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskMetricGroup.java @@ -139,24 +139,19 @@ public class TaskMetricGroup extends ComponentMetricGroup METRICS_OPERATOR_NAME_MAX_LENGTH) { LOG.warn("The operator name {} exceeded the {} characters length limit and was truncated.", name, METRICS_OPERATOR_NAME_MAX_LENGTH); - name = name.substring(0, METRICS_OPERATOR_NAME_MAX_LENGTH); + metricName = name.substring(0, METRICS_OPERATOR_NAME_MAX_LENGTH); + } else { + metricName = name; } - OperatorMetricGroup operator = new OperatorMetricGroup(this.registry, this, operatorID, name); + // unique OperatorIDs only exist in streaming, so we have to rely on the name for batch operators - final String key = operatorID + name; + final String key = operatorID + metricName; synchronized (this) { - OperatorMetricGroup previous = operators.put(key, operator); - if (previous == null) { - // no operator group so far - return operator; - } else { - // already had an operator group. restore that one. - operators.put(key, previous); - return previous; - } + return operators.computeIfAbsent(key, operator -> new OperatorMetricGroup(this.registry, this, operatorID, metricName)); } } -- Gitee From 1728e03a179235b70e6b6ad6532eb307717fd171 Mon Sep 17 00:00:00 2001 From: "Alec.Ch" Date: Sat, 21 Sep 2019 17:24:17 +0800 Subject: [PATCH 018/268] [FLINK-14119][table-planner-blink] Fix idle state is not cleaned for RetractableTopNFunction This closes #9741 --- .../rank/RetractableTopNFunction.java | 10 ++++++ .../rank/RetractableTopNFunctionTest.java | 34 +++++++++++++++++++ 2 files changed, 44 insertions(+) diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunction.java index 11820e0a26..5ed2c7b264 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunction.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunction.java @@ -117,6 +117,9 @@ public class RetractableTopNFunction extends AbstractTopNFunction { @Override public void processElement(BaseRow input, Context ctx, Collector out) throws Exception { + long currentTime = ctx.timerService().currentProcessingTime(); + // register state-cleanup timer + registerProcessingCleanupTimer(ctx, currentTime); initRankEnd(input); SortedMap sortedMap = treeMap.value(); if (sortedMap == null) { @@ -171,6 +174,13 @@ public class RetractableTopNFunction extends AbstractTopNFunction { treeMap.update(sortedMap); } + @Override + public void onTimer(long timestamp, OnTimerContext ctx, Collector out) throws Exception { + if (stateCleaningEnabled) { + cleanupState(dataState, treeMap); + } + } + // ------------- ROW_NUMBER------------------------------- private void retractRecordWithRowNumber( diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunctionTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunctionTest.java index e404b4186b..f852ca1384 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunctionTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunctionTest.java @@ -219,4 +219,38 @@ public class RetractableTopNFunctionTest extends TopNFunctionTestBase { .assertOutputEqualsSorted("output wrong.", expectedOutput, testHarness.getOutput()); } + @Test + public void testCleanIdleState() throws Exception { + AbstractTopNFunction func = createFunction(RankType.ROW_NUMBER, new ConstantRankRange(1, 2), true, + true); + OneInputStreamOperatorTestHarness testHarness = createTestHarness(func); + testHarness.open(); + // register cleanup timer with 20L + testHarness.setProcessingTime(0L); + testHarness.processElement(record("book", 1L, 12)); + testHarness.processElement(record("fruit", 5L, 22)); + + // register cleanup timer with 29L + testHarness.setProcessingTime(9L); + testHarness.processElement(retractRecord("book", 1L, 12)); + testHarness.processElement(record("fruit", 4L, 11)); + + // trigger the first cleanup timer and register cleanup timer with 4000 + testHarness.setProcessingTime(20L); + testHarness.processElement(record("fruit", 8L, 100)); + testHarness.processElement(record("book", 1L, 12)); + testHarness.close(); + + List expectedOutput = new ArrayList<>(); + expectedOutput.add(record("book", 1L, 12, 1L)); + expectedOutput.add(record("fruit", 5L, 22, 1L)); + expectedOutput.add(deleteRecord("book", 1L, 12, 1L)); + expectedOutput.add(deleteRecord("fruit", 5L, 22, 1L)); + expectedOutput.add(record("fruit", 5L, 22, 2L)); + expectedOutput.add(record("fruit", 4L, 11, 1L)); + // after idle state expired + expectedOutput.add(record("fruit", 8L, 100, 1L)); + expectedOutput.add(record("book", 1L, 12, 1L)); + assertorWithRowNumber.assertOutputEqualsSorted("output wrong.", expectedOutput, testHarness.getOutput()); + } } -- Gitee From de92c698bbba464d15de9b70546f1b1ceee5b48b Mon Sep 17 00:00:00 2001 From: Gyula Fora Date: Mon, 23 Sep 2019 15:26:42 +0200 Subject: [PATCH 019/268] [FLINK-14145] Fix getLatestCheckpoint(true) returns wrong checkpoint Closes #9727 --- .../checkpoint/CompletedCheckpointStore.java | 32 ++++----- ...tandaloneCompletedCheckpointStoreTest.java | 70 +++++++++++++++++-- 2 files changed, 78 insertions(+), 24 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java index 1cda131370..9ed3151742 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.util.FlinkRuntimeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,31 +54,26 @@ public interface CompletedCheckpointStore { * added. */ default CompletedCheckpoint getLatestCheckpoint(boolean isPreferCheckpointForRecovery) throws Exception { - if (getAllCheckpoints().isEmpty()) { + List allCheckpoints = getAllCheckpoints(); + if (allCheckpoints.isEmpty()) { return null; } - CompletedCheckpoint candidate = getAllCheckpoints().get(getAllCheckpoints().size() - 1); - if (isPreferCheckpointForRecovery && getAllCheckpoints().size() > 1) { - List allCheckpoints; - try { - allCheckpoints = getAllCheckpoints(); - ListIterator listIterator = allCheckpoints.listIterator(allCheckpoints.size() - 1); - while (listIterator.hasPrevious()) { - CompletedCheckpoint prev = listIterator.previous(); - if (!prev.getProperties().isSavepoint()) { - candidate = prev; - LOG.info("Found a completed checkpoint before the latest savepoint, will use it to recover!"); - break; - } + CompletedCheckpoint lastCompleted = allCheckpoints.get(allCheckpoints.size() - 1); + + if (isPreferCheckpointForRecovery && allCheckpoints.size() > 1 && lastCompleted.getProperties().isSavepoint()) { + ListIterator listIterator = allCheckpoints.listIterator(allCheckpoints.size() - 1); + while (listIterator.hasPrevious()) { + CompletedCheckpoint prev = listIterator.previous(); + if (!prev.getProperties().isSavepoint()) { + LOG.info("Found a completed checkpoint ({}) before the latest savepoint, will use it to recover!", prev); + return prev; } - } catch (Exception e) { - LOG.error("Method getAllCheckpoints caused exception : ", e); - throw new FlinkRuntimeException(e); } + LOG.info("Did not find earlier checkpoint, using latest savepoint to recover."); } - return candidate; + return lastCompleted; } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStoreTest.java index 6f3c60b5fc..4ed1050253 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStoreTest.java @@ -18,8 +18,10 @@ package org.apache.flink.runtime.checkpoint; +import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.state.SharedStateRegistry; + import org.junit.Test; import java.io.IOException; @@ -27,6 +29,8 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -41,7 +45,7 @@ public class StandaloneCompletedCheckpointStoreTest extends CompletedCheckpointS @Override protected CompletedCheckpointStore createCompletedCheckpoints( - int maxNumberOfCheckpointsToRetain) throws Exception { + int maxNumberOfCheckpointsToRetain) throws Exception { return new StandaloneCompletedCheckpointStore(maxNumberOfCheckpointsToRetain); } @@ -86,7 +90,7 @@ public class StandaloneCompletedCheckpointStoreTest extends CompletedCheckpointS assertTrue(checkpoint.isDiscarded()); verifyCheckpointDiscarded(taskStates); } - + /** * Tests that the checkpoint does not exist in the store when we fail to add * it into the store (i.e., there exists an exception thrown by the method). @@ -96,16 +100,16 @@ public class StandaloneCompletedCheckpointStoreTest extends CompletedCheckpointS final int numCheckpointsToRetain = 1; CompletedCheckpointStore store = createCompletedCheckpoints(numCheckpointsToRetain); - + for (long i = 0; i <= numCheckpointsToRetain; ++i) { CompletedCheckpoint checkpointToAdd = mock(CompletedCheckpoint.class); doReturn(i).when(checkpointToAdd).getCheckpointID(); doReturn(Collections.emptyMap()).when(checkpointToAdd).getOperatorStates(); doThrow(new IOException()).when(checkpointToAdd).discardOnSubsume(); - + try { store.addCheckpoint(checkpointToAdd); - + // The checkpoint should be in the store if we successfully add it into the store. List addedCheckpoints = store.getAllCheckpoints(); assertTrue(addedCheckpoints.contains(checkpointToAdd)); @@ -116,4 +120,60 @@ public class StandaloneCompletedCheckpointStoreTest extends CompletedCheckpointS } } } + + @Test + public void testPreferCheckpointWithoutSavepoint() throws Exception { + StandaloneCompletedCheckpointStore store = new StandaloneCompletedCheckpointStore(5); + JobID jobId = new JobID(); + store.addCheckpoint(checkpoint(jobId, 1L)); + store.addCheckpoint(checkpoint(jobId, 2L)); + store.addCheckpoint(checkpoint(jobId, 3L)); + + CompletedCheckpoint latestCheckpoint = store.getLatestCheckpoint(true); + + assertThat(latestCheckpoint.getCheckpointID(), equalTo(3L)); + } + + @Test + public void testPreferCheckpointWithSavepoint() throws Exception { + StandaloneCompletedCheckpointStore store = new StandaloneCompletedCheckpointStore(5); + JobID jobId = new JobID(); + store.addCheckpoint(checkpoint(jobId, 1L)); + store.addCheckpoint(savepoint(jobId, 2L)); + store.addCheckpoint(savepoint(jobId, 3L)); + + CompletedCheckpoint latestCheckpoint = store.getLatestCheckpoint(true); + + assertThat(latestCheckpoint.getCheckpointID(), equalTo(1L)); + } + + @Test + public void testPreferCheckpointWithOnlySavepoint() throws Exception { + StandaloneCompletedCheckpointStore store = new StandaloneCompletedCheckpointStore(5); + JobID jobId = new JobID(); + store.addCheckpoint(savepoint(jobId, 1L)); + store.addCheckpoint(savepoint(jobId, 2L)); + + CompletedCheckpoint latestCheckpoint = store.getLatestCheckpoint(true); + + assertThat(latestCheckpoint.getCheckpointID(), equalTo(2L)); + } + + private static CompletedCheckpoint checkpoint(JobID jobId, long checkpointId) { + return new TestCompletedCheckpoint( + jobId, + checkpointId, + checkpointId, + Collections.emptyMap(), + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE)); + } + + private static CompletedCheckpoint savepoint(JobID jobId, long checkpointId) { + return new TestCompletedCheckpoint( + jobId, + checkpointId, + checkpointId, + Collections.emptyMap(), + CheckpointProperties.forSavepoint()); + } } -- Gitee From b3722800586780670a4f3e059c9c36ce10b7c839 Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Tue, 24 Sep 2019 16:11:45 +0800 Subject: [PATCH 020/268] [FLINK-14182][core] Make TimeUtils able to parse duration string with plural form labels This closes #9754. --- .../java/org/apache/flink/util/TimeUtils.java | 48 ++++++++++++++----- .../org/apache/flink/util/TimeUtilsTest.java | 12 +++++ 2 files changed, 48 insertions(+), 12 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/util/TimeUtils.java b/flink-core/src/main/java/org/apache/flink/util/TimeUtils.java index 3c33329e59..a1e1b5dc36 100644 --- a/flink-core/src/main/java/org/apache/flink/util/TimeUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/TimeUtils.java @@ -23,6 +23,7 @@ import java.time.temporal.ChronoUnit; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Locale; import java.util.Map; import java.util.stream.Collectors; @@ -112,24 +113,47 @@ public class TimeUtils { */ private enum TimeUnit { - DAYS(ChronoUnit.DAYS, "d", "day"), - HOURS(ChronoUnit.HOURS, "h", "hour"), - MINUTES(ChronoUnit.MINUTES, "min", "minute"), - SECONDS(ChronoUnit.SECONDS, "s", "sec", "second"), - MILLISECONDS(ChronoUnit.MILLIS, "ms", "milli", "millisecond"), - MICROSECONDS(ChronoUnit.MICROS, "µs", "micro", "microsecond"), - NANOSECONDS(ChronoUnit.NANOS, "ns", "nano", "nanosecond"); + DAYS(ChronoUnit.DAYS, singular("d"), plural("day")), + HOURS(ChronoUnit.HOURS, singular("h"), plural("hour")), + MINUTES(ChronoUnit.MINUTES, singular("min"), plural("minute")), + SECONDS(ChronoUnit.SECONDS, singular("s"), plural("sec"), plural("second")), + MILLISECONDS(ChronoUnit.MILLIS, singular("ms"), plural("milli"), plural("millisecond")), + MICROSECONDS(ChronoUnit.MICROS, singular("µs"), plural("micro"), plural("microsecond")), + NANOSECONDS(ChronoUnit.NANOS, singular("ns"), plural("nano"), plural("nanosecond")); - private String[] labels; + private static final String PLURAL_SUFFIX = "s"; - private ChronoUnit unit; + private final List labels; - TimeUnit(ChronoUnit unit, String... labels) { + private final ChronoUnit unit; + + TimeUnit(ChronoUnit unit, String[]... labels) { this.unit = unit; - this.labels = labels; + this.labels = Arrays.stream(labels).flatMap(ls -> Arrays.stream(ls)).collect(Collectors.toList()); + } + + /** + * @param label the original label + * @return the singular format of the original label + */ + private static String[] singular(String label) { + return new String[] { + label + }; + } + + /** + * @param label the original label + * @return both the singular format and plural format of the original label + */ + private static String[] plural(String label) { + return new String[] { + label, + label + PLURAL_SUFFIX + }; } - public String[] getLabels() { + public List getLabels() { return labels; } diff --git a/flink-core/src/test/java/org/apache/flink/util/TimeUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/TimeUtilsTest.java index da35176b57..49b2e86d01 100644 --- a/flink-core/src/test/java/org/apache/flink/util/TimeUtilsTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/TimeUtilsTest.java @@ -32,7 +32,9 @@ public class TimeUtilsTest { public void testParseDurationNanos() { assertEquals(424562, TimeUtils.parseDuration("424562ns").getNano()); assertEquals(424562, TimeUtils.parseDuration("424562nano").getNano()); + assertEquals(424562, TimeUtils.parseDuration("424562nanos").getNano()); assertEquals(424562, TimeUtils.parseDuration("424562nanosecond").getNano()); + assertEquals(424562, TimeUtils.parseDuration("424562nanoseconds").getNano()); assertEquals(424562, TimeUtils.parseDuration("424562 ns").getNano()); } @@ -40,7 +42,9 @@ public class TimeUtilsTest { public void testParseDurationMicros() { assertEquals(565731 * 1000L, TimeUtils.parseDuration("565731µs").getNano()); assertEquals(565731 * 1000L, TimeUtils.parseDuration("565731micro").getNano()); + assertEquals(565731 * 1000L, TimeUtils.parseDuration("565731micros").getNano()); assertEquals(565731 * 1000L, TimeUtils.parseDuration("565731microsecond").getNano()); + assertEquals(565731 * 1000L, TimeUtils.parseDuration("565731microseconds").getNano()); assertEquals(565731 * 1000L, TimeUtils.parseDuration("565731 µs").getNano()); } @@ -49,14 +53,19 @@ public class TimeUtilsTest { assertEquals(1234, TimeUtils.parseDuration("1234").toMillis()); assertEquals(1234, TimeUtils.parseDuration("1234ms").toMillis()); assertEquals(1234, TimeUtils.parseDuration("1234milli").toMillis()); + assertEquals(1234, TimeUtils.parseDuration("1234millis").toMillis()); assertEquals(1234, TimeUtils.parseDuration("1234millisecond").toMillis()); + assertEquals(1234, TimeUtils.parseDuration("1234milliseconds").toMillis()); assertEquals(1234, TimeUtils.parseDuration("1234 ms").toMillis()); } @Test public void testParseDurationSeconds() { assertEquals(667766, TimeUtils.parseDuration("667766s").getSeconds()); + assertEquals(667766, TimeUtils.parseDuration("667766sec").getSeconds()); + assertEquals(667766, TimeUtils.parseDuration("667766secs").getSeconds()); assertEquals(667766, TimeUtils.parseDuration("667766second").getSeconds()); + assertEquals(667766, TimeUtils.parseDuration("667766seconds").getSeconds()); assertEquals(667766, TimeUtils.parseDuration("667766 s").getSeconds()); } @@ -64,6 +73,7 @@ public class TimeUtilsTest { public void testParseDurationMinutes() { assertEquals(7657623, TimeUtils.parseDuration("7657623min").toMinutes()); assertEquals(7657623, TimeUtils.parseDuration("7657623minute").toMinutes()); + assertEquals(7657623, TimeUtils.parseDuration("7657623minutes").toMinutes()); assertEquals(7657623, TimeUtils.parseDuration("7657623 min").toMinutes()); } @@ -71,6 +81,7 @@ public class TimeUtilsTest { public void testParseDurationHours() { assertEquals(987654, TimeUtils.parseDuration("987654h").toHours()); assertEquals(987654, TimeUtils.parseDuration("987654hour").toHours()); + assertEquals(987654, TimeUtils.parseDuration("987654hours").toHours()); assertEquals(987654, TimeUtils.parseDuration("987654 h").toHours()); } @@ -78,6 +89,7 @@ public class TimeUtilsTest { public void testParseDurationDays() { assertEquals(987654, TimeUtils.parseDuration("987654d").toDays()); assertEquals(987654, TimeUtils.parseDuration("987654day").toDays()); + assertEquals(987654, TimeUtils.parseDuration("987654days").toDays()); assertEquals(987654, TimeUtils.parseDuration("987654 d").toDays()); } -- Gitee From fee61e1c4ff08ff956735247c8fbd08239c9487d Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Tue, 24 Sep 2019 16:30:29 +0800 Subject: [PATCH 021/268] [FLINK-14168][runtime] Remove unused BootstrapTools#generateTaskManagerConfiguration This closes #9755. --- .../clusterframework/BootstrapTools.java | 37 ------------------- 1 file changed, 37 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java index da226718f4..75de58178d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java @@ -23,8 +23,6 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.util.NetUtils; @@ -53,7 +51,6 @@ import java.util.Map; import scala.Some; import scala.Tuple2; -import scala.concurrent.duration.FiniteDuration; import static org.apache.flink.configuration.ConfigOptions.key; @@ -268,40 +265,6 @@ public class BootstrapTools { } } - /** - * Generate a task manager configuration. - * @param baseConfig Config to start from. - * @param jobManagerHostname Job manager host name. - * @param jobManagerPort Port of the job manager. - * @param numSlots Number of slots to configure. - * @param registrationTimeout Timeout for registration - * @return TaskManager configuration - */ - public static Configuration generateTaskManagerConfiguration( - Configuration baseConfig, - String jobManagerHostname, - int jobManagerPort, - int numSlots, - FiniteDuration registrationTimeout) { - - Configuration cfg = cloneConfiguration(baseConfig); - - if (jobManagerHostname != null && !jobManagerHostname.isEmpty()) { - cfg.setString(JobManagerOptions.ADDRESS, jobManagerHostname); - } - - if (jobManagerPort > 0) { - cfg.setInteger(JobManagerOptions.PORT, jobManagerPort); - } - - cfg.setString(TaskManagerOptions.REGISTRATION_TIMEOUT, registrationTimeout.toString()); - if (numSlots != -1){ - cfg.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, numSlots); - } - - return cfg; - } - /** * Writes a Flink YAML config file from a Flink Configuration object. * @param cfg The Flink config -- Gitee From 44e86178831284524d815124dbb5360623358990 Mon Sep 17 00:00:00 2001 From: Wei Zhong Date: Mon, 23 Sep 2019 17:31:21 +0800 Subject: [PATCH 022/268] [FLINK-14167][python] Move python-related scripts from flink-dist to flink-python. This closes #9746. --- flink-dist/src/main/assemblies/bin.xml | 7 +++++++ .../bin/pyflink-gateway-server.sh | 0 .../main/flink-bin => flink-python}/bin/pyflink-shell.sh | 0 .../flink-bin => flink-python}/bin/pyflink-udf-runner.sh | 0 4 files changed, 7 insertions(+) rename {flink-dist/src/main/flink-bin => flink-python}/bin/pyflink-gateway-server.sh (100%) rename {flink-dist/src/main/flink-bin => flink-python}/bin/pyflink-shell.sh (100%) rename {flink-dist/src/main/flink-bin => flink-python}/bin/pyflink-udf-runner.sh (100%) diff --git a/flink-dist/src/main/assemblies/bin.xml b/flink-dist/src/main/assemblies/bin.xml index 1b65486515..8c549fbce8 100644 --- a/flink-dist/src/main/assemblies/bin.xml +++ b/flink-dist/src/main/assemblies/bin.xml @@ -115,6 +115,13 @@ under the License. 755 + + + ../flink-python/bin/ + bin + 0755 + + ../flink-table/flink-sql-client/bin/ diff --git a/flink-dist/src/main/flink-bin/bin/pyflink-gateway-server.sh b/flink-python/bin/pyflink-gateway-server.sh similarity index 100% rename from flink-dist/src/main/flink-bin/bin/pyflink-gateway-server.sh rename to flink-python/bin/pyflink-gateway-server.sh diff --git a/flink-dist/src/main/flink-bin/bin/pyflink-shell.sh b/flink-python/bin/pyflink-shell.sh similarity index 100% rename from flink-dist/src/main/flink-bin/bin/pyflink-shell.sh rename to flink-python/bin/pyflink-shell.sh diff --git a/flink-dist/src/main/flink-bin/bin/pyflink-udf-runner.sh b/flink-python/bin/pyflink-udf-runner.sh similarity index 100% rename from flink-dist/src/main/flink-bin/bin/pyflink-udf-runner.sh rename to flink-python/bin/pyflink-udf-runner.sh -- Gitee From c235253a101d21a089404dc20690b58640686083 Mon Sep 17 00:00:00 2001 From: Piyush Narang Date: Fri, 20 Sep 2019 14:48:10 -0400 Subject: [PATCH 023/268] [FLINK-14158] Update Mesos scheduler to make leaseOfferExpiration and declinedOfferRefuse duration configurable This closes #9737. --- .../generated/mesos_configuration.html | 10 +++++++ .../mesos/configuration/MesosOptions.java | 26 +++++++++++++++++ .../MesosResourceManager.java | 6 ++++ .../mesos/scheduler/TaskSchedulerBuilder.java | 5 ++++ .../mesos/scheduler/LaunchCoordinator.scala | 29 ++++++++++++++----- .../scheduler/LaunchCoordinatorTest.scala | 28 ++++++++++++++---- 6 files changed, 91 insertions(+), 13 deletions(-) diff --git a/docs/_includes/generated/mesos_configuration.html b/docs/_includes/generated/mesos_configuration.html index 9d5be9f358..9f951c52aa 100644 --- a/docs/_includes/generated/mesos_configuration.html +++ b/docs/_includes/generated/mesos_configuration.html @@ -27,6 +27,11 @@ true Enables SSL for the Flink artifact server. Note that security.ssl.enabled also needs to be set to true encryption to enable encryption. + +
mesos.resourcemanager.declined-offer-refuse-duration
+ 5000 + Amount of time to ask the Mesos master to not resend a declined resource offer again. This ensures a declined resource offer isn't resent immediately after being declined +
mesos.resourcemanager.framework.name
"Flink" @@ -57,5 +62,10 @@ (none) Comma-separated list of configuration keys which represent a configurable port. All port keys will dynamically get a port assigned through Mesos. + +
mesos.resourcemanager.unused-offer-expiration
+ 120000 + Amount of time to wait for unused expired offers before declining them. This ensures your scheduler will not hoard unuseful offers. + 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 51484c1d85..d957977ea8 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 @@ -118,4 +118,30 @@ public class MesosOptions { "All port keys will dynamically get a port assigned through Mesos.") .build()); + /** + * Config parameter to configure the amount of time to wait for unused expired Mesos + * offers before they are declined. + */ + public static final ConfigOption UNUSED_OFFER_EXPIRATION = + key("mesos.resourcemanager.unused-offer-expiration") + .defaultValue(120000L) + .withDescription( + Description.builder() + .text("Amount of time to wait for unused expired offers before declining them. " + + "This ensures your scheduler will not hoard unuseful offers.") + .build()); + + /** + * Config parameter to configure the amount of time refuse a particular offer for. + * This ensures the same resource offer isn't resent immediately after declining. + */ + public static final ConfigOption DECLINED_OFFER_REFUSE_DURATION = + key("mesos.resourcemanager.declined-offer-refuse-duration") + .defaultValue(5000L) + .withDescription( + Description.builder() + .text("Amount of time to ask the Mesos master to not resend a " + + "declined resource offer again. This ensures a declined resource offer " + + "isn't resent immediately after being declined") + .build()); } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java index 5cd8dc3449..01c68e4a0e 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java @@ -763,6 +763,12 @@ public class MesosResourceManager extends ResourceManagerFlinkTuple2} +import org.apache.flink.api.java.tuple.{Tuple2 => FlinkTuple2} import org.apache.flink.configuration.Configuration import org.apache.flink.mesos.Utils import org.apache.flink.mesos.scheduler.LaunchCoordinator._ import org.apache.flink.mesos.scheduler.messages._ import org.apache.flink.mesos.util.MesosResourceAllocation -import org.apache.mesos.{SchedulerDriver, Protos} +import org.apache.mesos.{Protos, SchedulerDriver} import scala.collection.JavaConverters._ import scala.collection.mutable.{Map => MutableMap} import scala.concurrent.duration._ +import org.apache.flink.mesos.configuration.MesosOptions._ /** * The launch coordinator handles offer processing, including @@ -54,6 +56,15 @@ class LaunchCoordinator( val LOG = Logger(getClass) + val declineOfferFilters: Protos.Filters = + Protos.Filters.newBuilder() + .setRefuseSeconds( + Duration(config.getLong(DECLINED_OFFER_REFUSE_DURATION), TimeUnit.MILLISECONDS).toSeconds) + .build() + + val unusedOfferExpirationDuration: Long = + Duration(config.getLong(UNUSED_OFFER_EXPIRATION), TimeUnit.MILLISECONDS).toSeconds + /** * The task placement optimizer. * @@ -68,13 +79,13 @@ class LaunchCoordinator( LOG.info(s"Declined offer ${lease.getId} from ${lease.hostname()} " + s"of memory ${lease.memoryMB()} MB, ${lease.cpuCores()} cpus, " + s"${lease.getScalarValue("gpus")} gpus, " - + s"of disk: ${lease.diskMB()} MB, network: ${lease.networkMbps()} Mbps") - schedulerDriver.declineOffer(lease.getOffer.getId) + + s"of disk: ${lease.diskMB()} MB, network: ${lease.networkMbps()} Mbps " + + s"for the next ${declineOfferFilters.getRefuseSeconds} seconds") + schedulerDriver.declineOffer(lease.getOffer.getId, declineOfferFilters) } }) - // avoid situations where we have lots of expired offers and we only expire a few at a time - .withRejectAllExpiredOffers() - .build + .withLeaseOfferExpirySecs(unusedOfferExpirationDuration) + .withRejectAllExpiredOffers().build } override def postStop(): Unit = { @@ -114,7 +125,9 @@ class LaunchCoordinator( case Event(offers: ResourceOffers, data: GatherData) => // decline any offers that come in schedulerDriver.suppressOffers() - for(offer <- offers.offers().asScala) { schedulerDriver.declineOffer(offer.getId) } + for(offer <- offers.offers().asScala) { + schedulerDriver.declineOffer(offer.getId, declineOfferFilters) + } stay() case Event(msg: Launch, data: GatherData) => diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala index 7b913ea789..00c23c6c14 100644 --- a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala +++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala @@ -18,6 +18,7 @@ package org.apache.flink.mesos.scheduler +import java.util.concurrent.TimeUnit import java.util.{Collections, UUID} import java.util.concurrent.atomic.AtomicReference @@ -27,13 +28,13 @@ import akka.testkit._ import com.netflix.fenzo.TaskRequest.{AssignedResources, NamedResourceSetRequest} import com.netflix.fenzo._ import com.netflix.fenzo.functions.{Action1, Action2} -import org.apache.flink.api.java.tuple.{Tuple2 => FlinkTuple2} +import org.apache.flink.api.java.tuple.{Tuple2=>FlinkTuple2} import org.apache.flink.configuration.Configuration import org.apache.flink.mesos.scheduler.LaunchCoordinator._ import org.apache.flink.mesos.scheduler.messages._ import org.apache.flink.runtime.akka.AkkaUtils import org.apache.mesos.Protos.{SlaveID, TaskInfo} -import org.apache.mesos.{Protos, SchedulerDriver} +import org.apache.mesos.{SchedulerDriver, Protos} import org.junit.runner.RunWith import org.mockito.Mockito.{verify, _} import org.mockito.invocation.InvocationOnMock @@ -47,8 +48,11 @@ import scala.collection.JavaConverters._ import org.apache.flink.mesos.Utils.range import org.apache.flink.mesos.Utils.ranges import org.apache.flink.mesos.Utils.scalar +import org.apache.flink.mesos.configuration.MesosOptions.DECLINED_OFFER_REFUSE_DURATION import org.apache.flink.mesos.util.MesosResourceAllocation +import scala.concurrent.duration.Duration + @RunWith(classOf[JUnitRunner]) class LaunchCoordinatorTest extends TestKitBase @@ -202,6 +206,9 @@ class LaunchCoordinatorTest def taskSchedulerBuilder(optimizer: TaskScheduler) = new TaskSchedulerBuilder { var leaseRejectAction: Action1[VirtualMachineLease] = null var rejectAllExpiredOffers: Boolean = false + var leaseOfferExpiry: Long = 0L + var offersToReject: Int = 0 + override def withLeaseRejectAction( action: Action1[VirtualMachineLease]): TaskSchedulerBuilder = { leaseRejectAction = action @@ -212,7 +219,13 @@ class LaunchCoordinatorTest this } + override def withLeaseOfferExpirySecs(leaseOfferExpirySecs: Long): TaskSchedulerBuilder = { + leaseOfferExpiry = leaseOfferExpirySecs + this + } + override def build(): TaskScheduler = optimizer + } /** @@ -239,6 +252,11 @@ class LaunchCoordinatorTest val trace = Mockito.inOrder(schedulerDriver) val fsm = TestFSMRef(new LaunchCoordinator(testActor, config, schedulerDriver, optimizerBuilder)) + val refuseFilter = + Protos.Filters.newBuilder() + .setRefuseSeconds( + Duration(config.getLong(DECLINED_OFFER_REFUSE_DURATION), TimeUnit.MILLISECONDS).toSeconds) + .build() val framework = randomFramework val task1 = randomTask @@ -321,8 +339,8 @@ class LaunchCoordinatorTest "stays in Idle with offers declined" in new Context { fsm.setState(Idle) fsm ! new ResourceOffers(Seq(slave1._3, slave1._4).asJava) - verify(schedulerDriver).declineOffer(slave1._3.getId) - verify(schedulerDriver).declineOffer(slave1._4.getId) + verify(schedulerDriver).declineOffer(slave1._3.getId, refuseFilter) + verify(schedulerDriver).declineOffer(slave1._4.getId, refuseFilter) fsm.stateName should be (Idle) } } @@ -462,7 +480,7 @@ class LaunchCoordinatorTest fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3)))) fsm ! StateTimeout - verify(schedulerDriver).declineOffer(slave1._3.getId) + verify(schedulerDriver).declineOffer(slave1._3.getId, refuseFilter) } } -- Gitee From 2773f98a64e447d1bdb02921dee24bdb35608621 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=A0=9A=E7=94=B0?= Date: Fri, 20 Sep 2019 15:18:37 +0800 Subject: [PATCH 024/268] [FLINK-13971][rest] Add TaskManager ID o.a.f.runtime.rest.messages.JobVertexTaskManagersInfo.TaskManagersInfo This closes #9724. --- docs/_includes/generated/rest_v1_dispatcher.html | 3 +++ .../src/test/resources/rest_api_v1.snapshot | 5 ++++- .../handler/job/JobVertexTaskManagersHandler.java | 13 +++++++++---- .../rest/messages/JobVertexTaskManagersInfo.java | 13 ++++++++++--- .../messages/JobVertexTaskManagersInfoTest.java | 2 +- 5 files changed, 27 insertions(+), 9 deletions(-) diff --git a/docs/_includes/generated/rest_v1_dispatcher.html b/docs/_includes/generated/rest_v1_dispatcher.html index 29ff267e1b..a3b91ee2e8 100644 --- a/docs/_includes/generated/rest_v1_dispatcher.html +++ b/docs/_includes/generated/rest_v1_dispatcher.html @@ -3509,6 +3509,9 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa "additionalProperties" : { "type" : "integer" } + }, + "taskmanager-id" : { + "type" : "string" } } } diff --git a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot index 629835d527..65345ba20d 100644 --- a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot +++ b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot @@ -1637,7 +1637,7 @@ "type" : "array", "items" : { "type" : "object", - "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:JobVertexDetailsInfo:SubtaskExecutionAttemptDetailsInfo", + "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:job:SubtaskExecutionAttemptDetailsInfo", "properties" : { "subtask" : { "type" : "integer" @@ -2317,6 +2317,9 @@ "additionalProperties" : { "type" : "integer" } + }, + "taskmanager-id" : { + "type" : "string" } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexTaskManagersHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexTaskManagersHandler.java index 183538a5f2..94906fe31d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexTaskManagersHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobVertexTaskManagersHandler.java @@ -107,12 +107,15 @@ public class JobVertexTaskManagersHandler extends AbstractExecutionGraphHandler< private static JobVertexTaskManagersInfo createJobVertexTaskManagersInfo(AccessExecutionJobVertex jobVertex, JobID jobID, @Nullable MetricFetcher metricFetcher) { // Build a map that groups tasks by TaskManager + Map taskManagerId2Host = new HashMap<>(); Map> taskManagerVertices = new HashMap<>(); for (AccessExecutionVertex vertex : jobVertex.getTaskVertices()) { TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation(); - String taskManager = location == null ? "(unassigned)" : location.getHostname() + ':' + location.dataPort(); + String taskManagerHost = location == null ? "(unassigned)" : location.getHostname() + ':' + location.dataPort(); + String taskmanagerId = location == null ? "(unassigned)" : location.getResourceID().toString(); + taskManagerId2Host.put(taskmanagerId, taskManagerHost); List vertices = taskManagerVertices.computeIfAbsent( - taskManager, + taskmanagerId, ignored -> new ArrayList<>(4)); vertices.add(vertex); } @@ -121,7 +124,8 @@ public class JobVertexTaskManagersHandler extends AbstractExecutionGraphHandler< List taskManagersInfoList = new ArrayList<>(4); for (Map.Entry> entry : taskManagerVertices.entrySet()) { - String host = entry.getKey(); + String taskmanagerId = entry.getKey(); + String host = taskManagerId2Host.get(taskmanagerId); List taskVertices = entry.getValue(); int[] tasksPerState = new int[ExecutionState.values().length]; @@ -193,7 +197,8 @@ public class JobVertexTaskManagersHandler extends AbstractExecutionGraphHandler< endTime, duration, jobVertexMetrics, - statusCounts)); + statusCounts, + taskmanagerId)); } return new JobVertexTaskManagersInfo(jobVertex.getJobVertexId(), jobVertex.getName(), now, taskManagersInfoList); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfo.java index 75ff570b37..df5014f976 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfo.java @@ -105,6 +105,7 @@ public class JobVertexTaskManagersInfo implements ResponseBody { public static final String TASK_MANAGERS_FIELD_DURATION = "duration"; public static final String TASK_MANAGERS_FIELD_METRICS = "metrics"; public static final String TASK_MANAGERS_FIELD_STATUS_COUNTS = "status-counts"; + public static final String TASK_MANAGERS_FIELD_TASKMANAGER_ID = "taskmanager-id"; @JsonProperty(TASK_MANAGERS_FIELD_HOST) private final String host; @@ -127,6 +128,9 @@ public class JobVertexTaskManagersInfo implements ResponseBody { @JsonProperty(TASK_MANAGERS_FIELD_STATUS_COUNTS) private final Map statusCounts; + @JsonProperty(TASK_MANAGERS_FIELD_TASKMANAGER_ID) + private final String taskmanagerId; + @JsonCreator public TaskManagersInfo( @JsonProperty(TASK_MANAGERS_FIELD_HOST) String host, @@ -135,7 +139,8 @@ public class JobVertexTaskManagersInfo implements ResponseBody { @JsonProperty(TASK_MANAGERS_FIELD_END_TIME) long endTime, @JsonProperty(TASK_MANAGERS_FIELD_DURATION) long duration, @JsonProperty(TASK_MANAGERS_FIELD_METRICS) IOMetricsInfo metrics, - @JsonProperty(TASK_MANAGERS_FIELD_STATUS_COUNTS) Map statusCounts) { + @JsonProperty(TASK_MANAGERS_FIELD_STATUS_COUNTS) Map statusCounts, + @JsonProperty(TASK_MANAGERS_FIELD_TASKMANAGER_ID) String taskmanagerId) { this.host = checkNotNull(host); this.status = checkNotNull(status); this.startTime = startTime; @@ -143,6 +148,7 @@ public class JobVertexTaskManagersInfo implements ResponseBody { this.duration = duration; this.metrics = checkNotNull(metrics); this.statusCounts = checkNotNull(statusCounts); + this.taskmanagerId = taskmanagerId; } @Override @@ -160,12 +166,13 @@ public class JobVertexTaskManagersInfo implements ResponseBody { endTime == that.endTime && duration == that.duration && Objects.equals(metrics, that.metrics) && - Objects.equals(statusCounts, that.statusCounts); + Objects.equals(statusCounts, that.statusCounts) && + Objects.equals(taskmanagerId, that.taskmanagerId); } @Override public int hashCode() { - return Objects.hash(host, status, startTime, endTime, duration, metrics, statusCounts); + return Objects.hash(host, status, startTime, endTime, duration, metrics, statusCounts, taskmanagerId); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfoTest.java index 1a7b521f8c..3fe60d0c1f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobVertexTaskManagersInfoTest.java @@ -58,7 +58,7 @@ public class JobVertexTaskManagersInfoTest extends RestResponseMarshallingTestBa for (ExecutionState executionState : ExecutionState.values()) { statusCounts.put(executionState, count++); } - taskManagersInfoList.add(new TaskManagersInfo("host1", ExecutionState.CANCELING, 1L, 2L, 3L, jobVertexMetrics, statusCounts)); + taskManagersInfoList.add(new TaskManagersInfo("host1", ExecutionState.CANCELING, 1L, 2L, 3L, jobVertexMetrics, statusCounts, "taskmanagerId")); return new JobVertexTaskManagersInfo(new JobVertexID(), "test", System.currentTimeMillis(), taskManagersInfoList); } -- Gitee From fe60f996e792680f74efea24e2916d013c24a044 Mon Sep 17 00:00:00 2001 From: tszkitlo40 Date: Sat, 21 Sep 2019 13:29:57 +0800 Subject: [PATCH 025/268] [FLINK-13746][e2e] Whitelist [Terror] to avoid end to end test failure in es 2.3.5 This closes #9738. --- 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 bdecb328dc..41e0eec92b 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -329,6 +329,7 @@ function check_logs_for_errors { | 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 -v "[Terror] modules" \ | grep -ic "error" || true) if [[ ${error_count} -gt 0 ]]; then echo "Found error in log files:" -- Gitee From 2bcf5094919a0b3e6db99efcea0b259826f59ad0 Mon Sep 17 00:00:00 2001 From: Jeff Martin Date: Sun, 22 Sep 2019 21:27:12 -0700 Subject: [PATCH 026/268] [FLINK-14076] Ensure CheckpointException can be deserialized on JobManager This closes #9742. --- .../cassandra/CassandraSinkBaseTest.java | 5 ++- .../kafka/FlinkKafkaProducer011ITCase.java | 7 ++-- .../kafka/FlinkKafkaProducerITCase.java | 6 ++-- .../org/apache/flink/util/ExceptionUtils.java | 32 +++++++++++++++++++ .../checkpoint/CheckpointException.java | 13 ++++++-- .../sink/TwoPhaseCommitSinkFunctionTest.java | 3 +- .../operators/AbstractStreamOperatorTest.java | 4 +-- .../flink/streaming/util/ContentDump.java | 2 +- 8 files changed, 59 insertions(+), 13 deletions(-) diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBaseTest.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBaseTest.java index b4406ab4d1..3ce9742c5f 100644 --- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBaseTest.java +++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraSinkBaseTest.java @@ -37,11 +37,13 @@ import org.junit.Test; import java.io.IOException; import java.time.Duration; import java.util.LinkedList; +import java.util.Optional; import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeoutException; +import static org.apache.flink.util.ExceptionUtils.findSerializedThrowable; import static org.hamcrest.number.OrderingComparison.greaterThan; import static org.mockito.Mockito.mock; import static org.powermock.api.mockito.PowerMockito.when; @@ -156,7 +158,8 @@ public class CassandraSinkBaseTest { Assert.fail(); } catch (Exception e) { - Assert.assertTrue(e.getCause() instanceof IOException); + Optional exCause = findSerializedThrowable(e, IOException.class, ClassLoader.getSystemClassLoader()); + Assert.assertTrue(exCause.isPresent()); } } } 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 0932d4260e..b2d0a96c3c 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 @@ -49,7 +49,7 @@ 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.ExceptionUtils.findSerializedThrowable; import static org.apache.flink.util.Preconditions.checkState; import static org.hamcrest.Matchers.lessThan; import static org.junit.Assert.assertThat; @@ -160,7 +160,7 @@ public class FlinkKafkaProducer011ITCase extends KafkaTestBaseWithFlink { } catch (Exception ex) { // testHarness1 will be fenced off after creating and closing testHarness2 - if (!findThrowable(ex, ProducerFencedException.class).isPresent()) { + if (!findSerializedThrowable(ex, ProducerFencedException.class, ClassLoader.getSystemClassLoader()).isPresent()) { throw ex; } } @@ -664,7 +664,8 @@ public class FlinkKafkaProducer011ITCase extends KafkaTestBaseWithFlink { } private boolean isCausedBy(FlinkKafka011ErrorCode expectedErrorCode, Throwable ex) { - Optional cause = findThrowable(ex, FlinkKafka011Exception.class); + // Extract the root cause kafka exception (if any) from the serialized throwable. + Optional cause = findSerializedThrowable(ex, FlinkKafka011Exception.class, ClassLoader.getSystemClassLoader()); if (cause.isPresent()) { return cause.get().getErrorCode().equals(expectedErrorCode); } 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 2242e811fa..3146af4763 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 @@ -45,7 +45,7 @@ import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.IntStream; -import static org.apache.flink.util.ExceptionUtils.findThrowable; +import static org.apache.flink.util.ExceptionUtils.findSerializedThrowable; import static org.apache.flink.util.Preconditions.checkState; import static org.hamcrest.Matchers.lessThan; import static org.junit.Assert.assertThat; @@ -156,7 +156,7 @@ public class FlinkKafkaProducerITCase extends KafkaTestBase { } catch (Exception ex) { // testHarness1 will be fenced off after creating and closing testHarness2 - if (!findThrowable(ex, ProducerFencedException.class).isPresent()) { + if (!findSerializedThrowable(ex, ProducerFencedException.class, ClassLoader.getSystemClassLoader()).isPresent()) { throw ex; } } @@ -662,7 +662,7 @@ public class FlinkKafkaProducerITCase extends KafkaTestBase { } private boolean isCausedBy(FlinkKafkaErrorCode expectedErrorCode, Throwable ex) { - Optional cause = findThrowable(ex, FlinkKafkaException.class); + Optional cause = findSerializedThrowable(ex, FlinkKafkaException.class, ClassLoader.getSystemClassLoader()); if (cause.isPresent()) { return cause.get().getErrorCode().equals(expectedErrorCode); } diff --git a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java index 721bf7f303..ddd0276a05 100644 --- a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java @@ -314,6 +314,38 @@ public final class ExceptionUtils { } } + /** + * Checks whether a throwable chain contains a specific type of exception and returns it. It deserializes + * any {@link SerializedThrowable} that are found using the provided {@link ClassLoader}. + * + * @param throwable the throwable chain to check. + * @param searchType the type of exception to search for in the chain. + * @param classLoader to use for deserialization. + * @return Optional throwable of the requested type if available, otherwise empty + */ + public static Optional findSerializedThrowable(Throwable throwable, Class searchType, ClassLoader classLoader) { + if (throwable == null || searchType == null) { + return Optional.empty(); + } + + Throwable t = throwable; + while (t != null) { + if (searchType.isAssignableFrom(t.getClass())) { + return Optional.of(searchType.cast(t)); + } else if (t.getClass().isAssignableFrom(SerializedThrowable.class)) { + Throwable next = ((SerializedThrowable) t).deserializeError(classLoader); + // SerializedThrowable#deserializeError returns itself under some conditions (e.g., null cause). + // If that happens, exit to avoid looping infinitely. This is ok because if the user was searching + // for a SerializedThrowable, we would have returned it in the initial if condition. + t = (next == t) ? null : next; + } else { + t = t.getCause(); + } + } + + return Optional.empty(); + } + /** * Checks whether a throwable chain contains a specific type of exception and returns it. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointException.java index c0bc2d1573..7c8ab49bfd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointException.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.SerializedThrowable; /** * Base class for checkpoint related exceptions. @@ -40,12 +41,20 @@ public class CheckpointException extends Exception { } public CheckpointException(CheckpointFailureReason failureReason, Throwable cause) { - super(failureReason.message(), cause); + // Defensively replace the cause with a SerializedThrowable in case it's a user-defined exception + // that doesn't exist on the JobManager's default classpath. + super( + failureReason.message(), + cause == null ? null : new SerializedThrowable(cause)); this.checkpointFailureReason = Preconditions.checkNotNull(failureReason); } public CheckpointException(String message, CheckpointFailureReason failureReason, Throwable cause) { - super(message + " Failure reason: " + failureReason.message(), cause); + // Defensively replace the cause with a SerializedThrowable in case it's a user-defined exception + // that doesn't exist on the JobManager's default classpath. + super( + message + " Failure reason: " + failureReason.message(), + cause == null ? null : new SerializedThrowable(cause)); this.checkpointFailureReason = Preconditions.checkNotNull(failureReason); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java index 2970b87789..84c01043c6 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java @@ -47,6 +47,7 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; +import static org.apache.flink.util.ExceptionUtils.findSerializedThrowable; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.hasItem; import static org.junit.Assert.assertEquals; @@ -167,7 +168,7 @@ public class TwoPhaseCommitSinkFunctionTest { harness.snapshot(2, 5); fail("something should fail"); } catch (Exception ex) { - if (!(ex.getCause() instanceof ContentDump.NotWritableException)) { + if (!findSerializedThrowable(ex, ContentDump.NotWritableException.class, ClassLoader.getSystemClassLoader()).isPresent()) { throw ex; } // ignore diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java index 36fb867872..f9d9aa5e50 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java @@ -558,7 +558,7 @@ public class AbstractStreamOperatorTest { new MemCheckpointStreamFactory(Integer.MAX_VALUE)); fail("Exception expected."); } catch (Exception e) { - assertEquals(failingException, e.getCause()); + assertEquals(failingException.getMessage(), e.getCause().getMessage()); } } @@ -636,7 +636,7 @@ public class AbstractStreamOperatorTest { new MemCheckpointStreamFactory(Integer.MAX_VALUE)); fail("Exception expected."); } catch (Exception e) { - assertEquals(failingException, e.getCause()); + assertEquals(failingException.getMessage(), e.getCause().getMessage()); } // verify that the context has been closed, the operator snapshot result has been cancelled diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/ContentDump.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/ContentDump.java index 903b237790..5c1568eeb2 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/ContentDump.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/ContentDump.java @@ -124,7 +124,7 @@ public class ContentDump { /** * Exception thrown for an attempt to write into read-only {@link ContentDump}. */ - public class NotWritableException extends RuntimeException { + public static class NotWritableException extends RuntimeException { public NotWritableException(String name) { super(String.format("File [%s] is not writable", name)); } -- Gitee From 67e73fad6cd80911377b2009e5e9c661f0324867 Mon Sep 17 00:00:00 2001 From: tison Date: Thu, 19 Sep 2019 16:29:29 +0800 Subject: [PATCH 027/268] [FLINK-14010][coordination] YarnResourceManager#onShutdownRequest triggers fatal error This closes #9719. --- .../flink/yarn/YarnResourceManager.java | 4 +++- .../flink/yarn/YarnResourceManagerTest.java | 20 +++++++++++++++++++ 2 files changed, 23 insertions(+), 1 deletion(-) 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 2e1f62bef0..8e2d83d0fc 100755 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -96,6 +96,8 @@ public class YarnResourceManager extends ResourceManager impleme * In task executor we use the hostnames given by YARN consistently throughout akka */ static final String ENV_FLINK_NODE_ID = "_FLINK_NODE_ID"; + static final String ERROR_MASSAGE_ON_SHUTDOWN_REQUEST = "Received shutdown request from YARN ResourceManager."; + /** Default heartbeat interval between this resource manager and the YARN ResourceManager. */ private final int yarnHeartbeatIntervalMillis; @@ -452,7 +454,7 @@ public class YarnResourceManager extends ResourceManager impleme @Override public void onShutdownRequest() { - closeAsync(); + onFatalError(new ResourceManagerException(ERROR_MASSAGE_ON_SHUTDOWN_REQUEST)); } @Override 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 df7d85b892..d8a106e96f 100755 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.JobLeaderIdService; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.SlotRequest; +import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.resourcemanager.utils.MockResourceManagerRuntimeServices; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -49,6 +50,7 @@ import org.apache.flink.runtime.taskexecutor.SlotStatus; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.RunnableWithException; @@ -84,6 +86,7 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import static org.apache.flink.yarn.YarnConfigKeys.ENV_APP_ID; import static org.apache.flink.yarn.YarnConfigKeys.ENV_CLIENT_HOME_DIR; @@ -92,7 +95,9 @@ import static org.apache.flink.yarn.YarnConfigKeys.ENV_FLINK_CLASSPATH; import static org.apache.flink.yarn.YarnConfigKeys.ENV_HADOOP_USER_NAME; import static org.apache.flink.yarn.YarnConfigKeys.FLINK_JAR_PATH; import static org.apache.flink.yarn.YarnConfigKeys.FLINK_YARN_FILES; +import static org.apache.flink.yarn.YarnResourceManager.ERROR_MASSAGE_ON_SHUTDOWN_REQUEST; import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -343,6 +348,21 @@ public class YarnResourceManagerTest extends TestLogger { return mockContainerStatus; } + @Test + public void testShutdownRequestCausesFatalError() throws Exception { + new Context() {{ + runTest(() -> { + resourceManager.onShutdownRequest(); + + Throwable t = testingFatalErrorHandler.getErrorFuture().get(2000L, TimeUnit.MILLISECONDS); + assertThat(ExceptionUtils.findThrowable(t, ResourceManagerException.class).isPresent(), is(true)); + assertThat(ExceptionUtils.findThrowableWithMessage(t, ERROR_MASSAGE_ON_SHUTDOWN_REQUEST).isPresent(), is(true)); + + testingFatalErrorHandler.clearError(); + }); + }}; + } + @Test public void testStopWorker() throws Exception { new Context() {{ -- Gitee From 02e26036e6d74e6c76ef1d793ac141c820715023 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Thu, 19 Sep 2019 22:15:26 +0800 Subject: [PATCH 028/268] [FLINK-14129][hive] HiveTableSource should implement ProjectableTableSource Implement ProjectableTableSource for HiveTableSource. This closes #9721. --- .../connectors/hive/HiveTableInputFormat.java | 57 ++++++++++++------- .../connectors/hive/HiveTableSource.java | 47 ++++++++++++--- .../connectors/hive/HiveTableSourceTest.java | 37 +++++++++++- 3 files changed, 109 insertions(+), 32 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableInputFormat.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableInputFormat.java index 8a38fb3555..f00288f900 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableInputFormat.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableInputFormat.java @@ -52,6 +52,7 @@ import java.io.ObjectOutputStream; import java.util.ArrayList; import java.util.List; import java.util.Properties; +import java.util.stream.IntStream; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR; @@ -73,9 +74,6 @@ public class HiveTableInputFormat extends HadoopInputFormatCommonBase partitionColNames; //For non-partition hive table, partitions only contains one partition which partitionValues is empty. @@ -88,17 +86,25 @@ public class HiveTableInputFormat extends HadoopInputFormatCommonBase partitions) { + List partitions, + int[] projectedFields) { super(jobConf.getCredentials()); checkNotNull(catalogTable, "catalogTable can not be null."); this.partitions = checkNotNull(partitions, "partitions can not be null."); this.jobConf = new JobConf(jobConf); this.partitionColNames = catalogTable.getPartitionKeys(); - rowArity = catalogTable.getSchema().getFieldCount(); + int rowArity = catalogTable.getSchema().getFieldCount(); + fields = projectedFields != null ? projectedFields : IntStream.range(0, rowArity).toArray(); } @Override @@ -137,6 +143,7 @@ public class HiveTableInputFormat extends HadoopInputFormatCommonBase= structFields.size()) { + String partition = partitionColNames.get(fields[i] - structFields.size()); + reuse.setField(i, hiveTablePartition.getPartitionSpec().get(partition)); + } + } + } + rowReused = true; + } this.fetched = false; - return row; + return reuse; } // -------------------------------------------------------------------------------------------- @@ -236,9 +253,9 @@ public class HiveTableInputFormat extends HadoopInputFormatCommonBase) in.readObject(); partitions = (List) in.readObject(); + fields = (int[]) in.readObject(); } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java index 451bd938a4..cfa1e629fe 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java @@ -29,6 +29,7 @@ import org.apache.flink.table.catalog.hive.client.HiveMetastoreClientWrapper; import org.apache.flink.table.catalog.hive.descriptors.HiveCatalogValidator; import org.apache.flink.table.sources.InputFormatTableSource; import org.apache.flink.table.sources.PartitionableTableSource; +import org.apache.flink.table.sources.ProjectableTableSource; import org.apache.flink.table.sources.TableSource; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalTypeRoot; @@ -45,6 +46,7 @@ import org.slf4j.LoggerFactory; import java.sql.Date; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -52,7 +54,7 @@ import java.util.Map; /** * A TableSource implementation to read data from Hive tables. */ -public class HiveTableSource extends InputFormatTableSource implements PartitionableTableSource { +public class HiveTableSource extends InputFormatTableSource implements PartitionableTableSource, ProjectableTableSource { private static Logger logger = LoggerFactory.getLogger(HiveTableSource.class); @@ -66,6 +68,7 @@ public class HiveTableSource extends InputFormatTableSource implements Part private Map, HiveTablePartition> partitionSpec2HiveTablePartition = new HashMap<>(); private boolean initAllPartitions; private boolean partitionPruned; + private int[] projectedFields; public HiveTableSource(JobConf jobConf, ObjectPath tablePath, CatalogTable catalogTable) { this.jobConf = Preconditions.checkNotNull(jobConf); @@ -77,18 +80,23 @@ public class HiveTableSource extends InputFormatTableSource implements Part partitionPruned = false; } + // A constructor mainly used to create copies during optimizations like partition pruning and projection push down. private HiveTableSource(JobConf jobConf, ObjectPath tablePath, CatalogTable catalogTable, List allHivePartitions, String hiveVersion, - List> partitionList) { + List> partitionList, + boolean initAllPartitions, + boolean partitionPruned, + int[] projectedFields) { this.jobConf = Preconditions.checkNotNull(jobConf); this.tablePath = Preconditions.checkNotNull(tablePath); this.catalogTable = Preconditions.checkNotNull(catalogTable); this.allHivePartitions = allHivePartitions; this.hiveVersion = hiveVersion; this.partitionList = partitionList; - this.initAllPartitions = true; - partitionPruned = true; + this.initAllPartitions = initAllPartitions; + this.partitionPruned = partitionPruned; + this.projectedFields = projectedFields; } @Override @@ -96,7 +104,7 @@ public class HiveTableSource extends InputFormatTableSource implements Part if (!initAllPartitions) { initAllPartitions(); } - return new HiveTableInputFormat(jobConf, catalogTable, allHivePartitions); + return new HiveTableInputFormat(jobConf, catalogTable, allHivePartitions, projectedFields); } @Override @@ -106,7 +114,17 @@ public class HiveTableSource extends InputFormatTableSource implements Part @Override public DataType getProducedDataType() { - return getTableSchema().toRowDataType(); + TableSchema originSchema = getTableSchema(); + if (projectedFields == null) { + return originSchema.toRowDataType(); + } + String[] names = new String[projectedFields.length]; + DataType[] types = new DataType[projectedFields.length]; + for (int i = 0; i < projectedFields.length; i++) { + names[i] = originSchema.getFieldName(projectedFields[i]).get(); + types[i] = originSchema.getFieldDataType(projectedFields[i]).get(); + } + return TableSchema.builder().fields(names, types).build().toRowDataType(); } @Override @@ -140,7 +158,8 @@ public class HiveTableSource extends InputFormatTableSource implements Part "partition spec %s", partitionSpec)); remainingHivePartitions.add(hiveTablePartition); } - return new HiveTableSource(jobConf, tablePath, catalogTable, remainingHivePartitions, hiveVersion, partitionList); + return new HiveTableSource(jobConf, tablePath, catalogTable, remainingHivePartitions, + hiveVersion, partitionList, true, true, projectedFields); } } @@ -223,7 +242,17 @@ public class HiveTableSource extends InputFormatTableSource implements Part @Override public String explainSource() { - return super.explainSource() + String.format(" TablePath: %s, PartitionPruned: %s, PartitionNums: %d", - tablePath.getFullName(), partitionPruned, null == allHivePartitions ? 0 : allHivePartitions.size()); + String explain = String.format(" TablePath: %s, PartitionPruned: %s, PartitionNums: %d", + tablePath.getFullName(), partitionPruned, null == allHivePartitions ? 0 : allHivePartitions.size()); + if (projectedFields != null) { + explain += ", ProjectedFields: " + Arrays.toString(projectedFields); + } + return super.explainSource() + explain; + } + + @Override + public TableSource projectFields(int[] fields) { + return new HiveTableSource(jobConf, tablePath, catalogTable, allHivePartitions, hiveVersion, + partitionList, initAllPartitions, partitionPruned, fields); } } diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceTest.java index 3920d0f995..878cd686f7 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceTest.java @@ -181,9 +181,9 @@ public class HiveTableSourceTest { String abstractSyntaxTree = explain[1]; String optimizedLogicalPlan = explain[2]; String physicalExecutionPlan = explain[3]; - assertTrue(abstractSyntaxTree.contains("HiveTableSource(year, value, pt) TablePath: source_db.test_table_pt_1, PartitionPruned: false, PartitionNums: 2]")); - assertTrue(optimizedLogicalPlan.contains("HiveTableSource(year, value, pt) TablePath: source_db.test_table_pt_1, PartitionPruned: true, PartitionNums: 1]")); - assertTrue(physicalExecutionPlan.contains("HiveTableSource(year, value, pt) TablePath: source_db.test_table_pt_1, PartitionPruned: true, PartitionNums: 1]")); + assertTrue(abstractSyntaxTree.contains("HiveTableSource(year, value, pt) TablePath: source_db.test_table_pt_1, PartitionPruned: false, PartitionNums: 2")); + assertTrue(optimizedLogicalPlan.contains("HiveTableSource(year, value, pt) TablePath: source_db.test_table_pt_1, PartitionPruned: true, PartitionNums: 1")); + assertTrue(physicalExecutionPlan.contains("HiveTableSource(year, value, pt) TablePath: source_db.test_table_pt_1, PartitionPruned: true, PartitionNums: 1")); // second check execute results List rows = JavaConverters.seqAsJavaListConverter(TableUtil.collect((TableImpl) src)).asJava(); assertEquals(2, rows.size()); @@ -191,4 +191,35 @@ public class HiveTableSourceTest { assertArrayEquals(new String[]{"2014,3,0", "2014,4,0"}, rowStrings); } + @Test + public void testProjectionPushDown() throws Exception { + hiveShell.execute("create table src(x int,y string) partitioned by (p1 bigint, p2 string)"); + final String catalogName = "hive"; + try { + hiveShell.insertInto("default", "src") + .addRow(1, "a", 2013, "2013") + .addRow(2, "b", 2013, "2013") + .addRow(3, "c", 2014, "2014") + .commit(); + TableEnvironment tableEnv = HiveTestUtils.createTableEnv(); + tableEnv.registerCatalog(catalogName, hiveCatalog); + Table table = tableEnv.sqlQuery("select p1, count(y) from hive.`default`.src group by p1"); + String[] explain = tableEnv.explain(table).split("==.*==\n"); + assertEquals(4, explain.length); + String logicalPlan = explain[2]; + String physicalPlan = explain[3]; + String expectedExplain = + "HiveTableSource(x, y, p1, p2) TablePath: default.src, PartitionPruned: false, PartitionNums: 2, ProjectedFields: [2, 1]"; + assertTrue(logicalPlan.contains(expectedExplain)); + assertTrue(physicalPlan.contains(expectedExplain)); + + List rows = JavaConverters.seqAsJavaListConverter(TableUtil.collect((TableImpl) table)).asJava(); + assertEquals(2, rows.size()); + Object[] rowStrings = rows.stream().map(Row::toString).sorted().toArray(); + assertArrayEquals(new String[]{"2013,2", "2014,1"}, rowStrings); + } finally { + hiveShell.execute("drop table src"); + } + } + } -- Gitee From da7484a4c2c4b659cd01a18f62032aa85f709d69 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Tue, 10 Sep 2019 15:15:08 +0800 Subject: [PATCH 029/268] [hotfix][task] Refactor the class name AvailabilityListener to AvailabilityProvider --- ...{AvailabilityListener.java => AvailabilityProvider.java} | 6 +++--- .../org/apache/flink/runtime/io/NullableAsyncDataInput.java | 2 +- .../org/apache/flink/runtime/io/PullingAsyncDataInput.java | 2 +- .../flink/streaming/runtime/io/PushingAsyncDataInput.java | 4 ++-- .../flink/streaming/runtime/io/StreamInputProcessor.java | 4 ++-- 5 files changed, 9 insertions(+), 9 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/io/{AvailabilityListener.java => AvailabilityProvider.java} (93%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityProvider.java similarity index 93% rename from flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityListener.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityProvider.java index f7e61d2d8d..4956cfbef3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityListener.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityProvider.java @@ -26,7 +26,7 @@ import java.util.concurrent.CompletableFuture; * {@link CompletableFuture}. For usage check out for example {@link PullingAsyncDataInput}. */ @Internal -public interface AvailabilityListener { +public interface AvailabilityProvider { /** * Constant that allows to avoid volatile checks {@link CompletableFuture#isDone()}. Check * {@link #isAvailable()} for more explanation. @@ -45,8 +45,8 @@ public interface AvailabilityListener { * this method should do the following check: *
 	 * {@code
-	 *	AvailabilityListener input = ...;
-	 *	if (input.isAvailable() == AvailabilityListener.AVAILABLE || input.isAvailable().isDone()) {
+	 *	AvailabilityProvider input = ...;
+	 *	if (input.isAvailable() == AvailabilityProvider.AVAILABLE || input.isAvailable().isDone()) {
 	 *		// do something;
 	 *	}
 	 * }
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/NullableAsyncDataInput.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/NullableAsyncDataInput.java
index 3c65e4413c..4b00955479 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/NullableAsyncDataInput.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/NullableAsyncDataInput.java
@@ -27,7 +27,7 @@ import javax.annotation.Nullable;
  * {@link PullingAsyncDataInput#pollNext()}.
  */
 @Internal
-public interface NullableAsyncDataInput extends AvailabilityListener {
+public interface NullableAsyncDataInput extends AvailabilityProvider {
 	/**
 	 * Poll the next element. This method should be non blocking.
 	 *
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/PullingAsyncDataInput.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/PullingAsyncDataInput.java
index f792c280a4..e27388e205 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/PullingAsyncDataInput.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/PullingAsyncDataInput.java
@@ -50,7 +50,7 @@ import java.util.concurrent.CompletableFuture;
  * 
*/ @Internal -public interface PullingAsyncDataInput extends AvailabilityListener { +public interface PullingAsyncDataInput extends AvailabilityProvider { /** * Poll the next element. This method should be non blocking. * diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/PushingAsyncDataInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/PushingAsyncDataInput.java index b912370ce0..7fcd047e0a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/PushingAsyncDataInput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/PushingAsyncDataInput.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.runtime.io; import org.apache.flink.annotation.Internal; -import org.apache.flink.runtime.io.AvailabilityListener; +import org.apache.flink.runtime.io.AvailabilityProvider; import org.apache.flink.runtime.io.PullingAsyncDataInput; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; @@ -31,7 +31,7 @@ import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; * of returning {@code Optional.empty()} via {@link PullingAsyncDataInput#pollNext()}. */ @Internal -public interface PushingAsyncDataInput extends AvailabilityListener { +public interface PushingAsyncDataInput extends AvailabilityProvider { /** * Pushes the next element to the output from current data input, and returns diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java index 1de31bf6df..4fef008234 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java @@ -19,7 +19,7 @@ package org.apache.flink.streaming.runtime.io; import org.apache.flink.annotation.Internal; -import org.apache.flink.runtime.io.AvailabilityListener; +import org.apache.flink.runtime.io.AvailabilityProvider; import java.io.Closeable; @@ -27,7 +27,7 @@ import java.io.Closeable; * Interface for processing records by {@link org.apache.flink.streaming.runtime.tasks.StreamTask}. */ @Internal -public interface StreamInputProcessor extends AvailabilityListener, Closeable { +public interface StreamInputProcessor extends AvailabilityProvider, Closeable { /** * @return true if {@link StreamInputProcessor} estimates that more records can be processed * immediately. Otherwise false, which means that there are no more records available at the -- Gitee From ba07cefdf9e5ce6cc823b34c9dd1b00a65b107d8 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Tue, 3 Sep 2019 14:59:03 +0800 Subject: [PATCH 030/268] [hotfix][task] Remove invalid condition in StreamTwoInputProcessor#isAnyInputAvailable When StreamTwoInputProcessor#processInput is called, that means at-least one input has not finished yet. So in the process of StreamTwoInputProcessor#isAnyInputAvailable, when the input1 is finished it is no need to check whether the input2 is finished or not. We could return the status of input2 directly. --- .../flink/streaming/runtime/io/StreamTwoInputProcessor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index 1e5d31fe88..e0b101d7b0 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -279,7 +279,7 @@ public final class StreamTwoInputProcessor implements StreamInputProce private CompletableFuture isAnyInputAvailable() { if (input1.isFinished()) { - return input2.isFinished() ? AVAILABLE : input2.isAvailable(); + return input2.isAvailable(); } if (input2.isFinished()) { -- Gitee From 9a8344e2562cffb606338101746bc3eafc40875e Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Thu, 12 Sep 2019 15:16:11 +0800 Subject: [PATCH 031/268] [hotfix][task] Refactor to integrate the logics of TwoInputSelectionHandler#setAvailable/UnavailableIndex Currently the logic of TwoInputSelectionHandler#setAvailableIndex is done before selectNextInputIndex, and the logic of TwoInputSelectionHandler#setUnavailableIndex is done after selectNextInputIndex. Actually these two logics are close with each other and can be done together, and it is helpful to trace the logics to avoid spreading anywhere. --- .../runtime/io/StreamTwoInputProcessor.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index e0b101d7b0..f3b19e7d77 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -184,10 +184,6 @@ public final class StreamTwoInputProcessor implements StreamInputProce } checkFinished(status, lastReadInputIndex); - if (status != InputStatus.MORE_AVAILABLE) { - inputSelectionHandler.setUnavailableInput(readingInputIndex); - } - return status == InputStatus.MORE_AVAILABLE; } @@ -253,11 +249,15 @@ public final class StreamTwoInputProcessor implements StreamInputProce } private void updateAvailability() { - if (!input1.isFinished() && input1.isAvailable() == AVAILABLE) { - inputSelectionHandler.setAvailableInput(input1.getInputIndex()); - } - if (!input2.isFinished() && input2.isAvailable() == AVAILABLE) { - inputSelectionHandler.setAvailableInput(input2.getInputIndex()); + updateAvailability(input1); + updateAvailability(input2); + } + + private void updateAvailability(StreamTaskInput input) { + if (!input.isFinished() && input.isAvailable() == AVAILABLE) { + inputSelectionHandler.setAvailableInput(input.getInputIndex()); + } else { + inputSelectionHandler.setUnavailableInput(input.getInputIndex()); } } -- Gitee From f41bda61f4dc73f3625ccb7347fa9c2db4d298df Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Thu, 12 Sep 2019 17:32:44 +0800 Subject: [PATCH 032/268] [FLINK-13767][task] Refactor StreamInputProcessor#processInput based on InputStatus StreamInputProcessor#processInput could return InputStatus instead of current boolean value to keep consistent with PushingAsyncDataInput#emitNext. For the implementation of StreamTwoInputProcessor#processInput, we could maintain and judge the two input status together with the next selected input index to determine the final precise status. To do so we could avoid invalid processInput call except for the first call. In addition, AvailabilityProvider#isFinished has the duplicated semantic with InputStatus#END_OF_INPUT for PushingAsyncDataInput, and it is only meaningful for PullingAsyncDataInput now. So we migrate the #isFinished method from AvailabilityProvider to PullingAsyncDataInput. --- .../runtime/io/AvailabilityProvider.java | 7 +- .../runtime/io/PullingAsyncDataInput.java | 5 + .../runtime/io/StreamInputProcessor.java | 8 +- .../runtime/io/StreamOneInputProcessor.java | 9 +- .../runtime/io/StreamTaskNetworkInput.java | 8 -- .../runtime/io/StreamTwoInputProcessor.java | 93 ++++++++++++------- .../streaming/runtime/tasks/StreamTask.java | 16 ++-- .../runtime/tasks/StreamTaskTest.java | 10 +- 8 files changed, 81 insertions(+), 75 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityProvider.java index 4956cfbef3..42e09366de 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/AvailabilityProvider.java @@ -33,11 +33,6 @@ public interface AvailabilityProvider { */ CompletableFuture AVAILABLE = CompletableFuture.completedFuture(null); - /** - * @return true if is finished and for example end of input was reached, false otherwise. - */ - boolean isFinished(); - /** * Check if this instance is available for further processing. * @@ -55,7 +50,7 @@ public interface AvailabilityProvider { * @return a future that is completed if there are more records available. If there are more * records available immediately, {@link #AVAILABLE} should be returned. Previously returned * not completed futures should become completed once there is more input available or if - * the input {@link #isFinished()}. + * the input is finished. */ CompletableFuture isAvailable(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/PullingAsyncDataInput.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/PullingAsyncDataInput.java index e27388e205..17109ccebb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/PullingAsyncDataInput.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/PullingAsyncDataInput.java @@ -58,4 +58,9 @@ public interface PullingAsyncDataInput extends AvailabilityProvider { * if {@link #isFinished()} returns true. Otherwise {@code Optional.of(element)}. */ Optional pollNext() throws Exception; + + /** + * @return true if is finished and for example end of input was reached, false otherwise. + */ + boolean isFinished(); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java index 4fef008234..3c070e28ef 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java @@ -29,9 +29,9 @@ import java.io.Closeable; @Internal public interface StreamInputProcessor extends AvailabilityProvider, Closeable { /** - * @return true if {@link StreamInputProcessor} estimates that more records can be processed - * immediately. Otherwise false, which means that there are no more records available at the - * moment and the caller should check {@link #isFinished()} and/or {@link #isAvailable()}. + * @return input status to estimate whether more records can be processed immediately or not. + * If there are no more records available at the moment and the caller should check finished + * state and/or {@link #isAvailable()}. */ - boolean processInput() throws Exception; + InputStatus processInput() throws Exception; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java index 4c21791ff5..ddeaf7be04 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamOneInputProcessor.java @@ -59,18 +59,13 @@ public final class StreamOneInputProcessor implements StreamInputProcessor { this.operatorChain = checkNotNull(operatorChain); } - @Override - public boolean isFinished() { - return input.isFinished(); - } - @Override public CompletableFuture isAvailable() { return input.isAvailable(); } @Override - public boolean processInput() throws Exception { + public InputStatus processInput() throws Exception { InputStatus status = input.emitNext(output); if (status == InputStatus.END_OF_INPUT) { @@ -79,7 +74,7 @@ public final class StreamOneInputProcessor implements StreamInputProcessor { } } - return status == InputStatus.MORE_AVAILABLE; + return status; } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java index 8f85709f26..ecb26ac061 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInput.java @@ -74,8 +74,6 @@ public final class StreamTaskNetworkInput implements StreamTaskInput { private RecordDeserializer> currentRecordDeserializer = null; - private boolean isFinished = false; - @SuppressWarnings("unchecked") public StreamTaskNetworkInput( CheckpointedInputGate checkpointedInputGate, @@ -137,7 +135,6 @@ public final class StreamTaskNetworkInput implements StreamTaskInput { processBufferOrEvent(bufferOrEvent.get()); } else { if (checkpointedInputGate.isFinished()) { - isFinished = true; checkState(checkpointedInputGate.isAvailable().isDone(), "Finished BarrierHandler should be available"); if (!checkpointedInputGate.isEmpty()) { throw new IllegalStateException("Trailing data in checkpoint barrier handler."); @@ -192,11 +189,6 @@ public final class StreamTaskNetworkInput implements StreamTaskInput { return inputIndex; } - @Override - public boolean isFinished() { - return isFinished; - } - @Override public CompletableFuture isAvailable() { if (currentRecordDeserializer != null) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index f3b19e7d77..24ddc244a0 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -62,6 +62,10 @@ public final class StreamTwoInputProcessor implements StreamInputProce private final DataOutput output1; private final DataOutput output2; + /** Input status to keep track for determining whether the input is finished or not. */ + private InputStatus firstInputStatus = InputStatus.MORE_AVAILABLE; + private InputStatus secondInputStatus = InputStatus.MORE_AVAILABLE; + /** * Stream status for the two inputs. We need to keep track for determining when * to forward stream status changes downstream. @@ -147,11 +151,6 @@ public final class StreamTwoInputProcessor implements StreamInputProce inputSelectionHandler.nextSelection(); } - @Override - public boolean isFinished() { - return input1.isFinished() && input2.isFinished(); - } - @Override public CompletableFuture isAvailable() { if (inputSelectionHandler.areAllInputsSelected()) { @@ -163,28 +162,42 @@ public final class StreamTwoInputProcessor implements StreamInputProce } @Override - public boolean processInput() throws Exception { - if (!isPrepared) { + public InputStatus processInput() throws Exception { + int readingInputIndex; + if (isPrepared) { + readingInputIndex = selectNextReadingInputIndex(); + assert readingInputIndex != -1; + } else { // the preparations here are not placed in the constructor because all work in it // must be executed after all operators are opened. - prepareForProcessing(); + readingInputIndex = selectFirstReadingInputIndex(); + if (readingInputIndex == -1) { + return InputStatus.NOTHING_AVAILABLE; + } } - int readingInputIndex = selectNextReadingInputIndex(); - if (readingInputIndex == -1) { - return false; - } lastReadInputIndex = readingInputIndex; - InputStatus status; if (readingInputIndex == 0) { - status = input1.emitNext(output1); + firstInputStatus = input1.emitNext(output1); + checkFinished(firstInputStatus, lastReadInputIndex); } else { - status = input2.emitNext(output2); + secondInputStatus = input2.emitNext(output2); + checkFinished(secondInputStatus, lastReadInputIndex); } - checkFinished(status, lastReadInputIndex); - return status == InputStatus.MORE_AVAILABLE; + return getInputStatus(); + } + + private int selectFirstReadingInputIndex() throws IOException { + // Note: the first call to nextSelection () on the operator must be made after this operator + // is opened to ensure that any changes about the input selection in its open() + // method take effect. + inputSelectionHandler.nextSelection(); + + isPrepared = true; + + return selectNextReadingInputIndex(); } private void checkFinished(InputStatus status, int inputIndex) throws Exception { @@ -196,6 +209,24 @@ public final class StreamTwoInputProcessor implements StreamInputProce } } + private InputStatus getInputStatus() { + if (firstInputStatus == InputStatus.END_OF_INPUT && secondInputStatus == InputStatus.END_OF_INPUT) { + return InputStatus.END_OF_INPUT; + } + + if (inputSelectionHandler.areAllInputsSelected()) { + if (firstInputStatus == InputStatus.MORE_AVAILABLE || secondInputStatus == InputStatus.MORE_AVAILABLE) { + return InputStatus.MORE_AVAILABLE; + } else { + return InputStatus.NOTHING_AVAILABLE; + } + } + + InputStatus selectedStatus = inputSelectionHandler.isFirstInputSelected() ? firstInputStatus : secondInputStatus; + InputStatus otherStatus = inputSelectionHandler.isFirstInputSelected() ? secondInputStatus : firstInputStatus; + return selectedStatus == InputStatus.END_OF_INPUT ? otherStatus : selectedStatus; + } + @Override public void close() throws IOException { IOException ex = null; @@ -240,49 +271,41 @@ public final class StreamTwoInputProcessor implements StreamInputProce if (inputSelectionHandler.areAllInputsSelected()) { return; } - if (inputSelectionHandler.isFirstInputSelected() && input1.isFinished()) { + if (inputSelectionHandler.isFirstInputSelected() && firstInputStatus == InputStatus.END_OF_INPUT) { throw new IOException("Can not make a progress: only first input is selected but it is already finished"); } - if (inputSelectionHandler.isSecondInputSelected() && input2.isFinished()) { + if (inputSelectionHandler.isSecondInputSelected() && secondInputStatus == InputStatus.END_OF_INPUT) { throw new IOException("Can not make a progress: only second input is selected but it is already finished"); } } private void updateAvailability() { - updateAvailability(input1); - updateAvailability(input2); + updateAvailability(firstInputStatus, input1); + updateAvailability(secondInputStatus, input2); } - private void updateAvailability(StreamTaskInput input) { - if (!input.isFinished() && input.isAvailable() == AVAILABLE) { + private void updateAvailability(InputStatus status, StreamTaskInput input) { + if (status == InputStatus.MORE_AVAILABLE || (status != InputStatus.END_OF_INPUT && input.isAvailable() == AVAILABLE)) { inputSelectionHandler.setAvailableInput(input.getInputIndex()); } else { inputSelectionHandler.setUnavailableInput(input.getInputIndex()); } } - private void prepareForProcessing() { - // Note: the first call to nextSelection () on the operator must be made after this operator - // is opened to ensure that any changes about the input selection in its open() - // method take effect. - inputSelectionHandler.nextSelection(); - - isPrepared = true; - } - private void checkAndSetAvailable(int inputIndex) { StreamTaskInput input = getInput(inputIndex); - if (!input.isFinished() && input.isAvailable().isDone()) { + InputStatus status = (inputIndex == 0 ? firstInputStatus : secondInputStatus); + if (status != InputStatus.END_OF_INPUT && input.isAvailable().isDone()) { inputSelectionHandler.setAvailableInput(inputIndex); } } private CompletableFuture isAnyInputAvailable() { - if (input1.isFinished()) { + if (firstInputStatus == InputStatus.END_OF_INPUT) { return input2.isAvailable(); } - if (input2.isFinished()) { + if (secondInputStatus == InputStatus.END_OF_INPUT) { return input1.isAvailable(); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 313beb401e..148a43d3c8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -58,6 +58,7 @@ import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl; +import org.apache.flink.streaming.runtime.io.InputStatus; import org.apache.flink.streaming.runtime.io.RecordWriterOutput; import org.apache.flink.streaming.runtime.io.StreamInputProcessor; import org.apache.flink.streaming.runtime.partitioner.ConfigurableStreamPartitioner; @@ -279,14 +280,13 @@ public abstract class StreamTask> * @throws Exception on any problems in the action. */ protected void processInput(DefaultActionContext context) throws Exception { - if (!inputProcessor.processInput()) { - if (inputProcessor.isFinished()) { - context.allActionsCompleted(); - } - else { - SuspendedMailboxDefaultAction suspendedDefaultAction = context.suspendDefaultAction(); - inputProcessor.isAvailable().thenRun(suspendedDefaultAction::resume); - } + InputStatus status = inputProcessor.processInput(); + if (status == InputStatus.END_OF_INPUT) { + context.allActionsCompleted(); + } + else if (status == InputStatus.NOTHING_AVAILABLE){ + SuspendedMailboxDefaultAction suspendedDefaultAction = context.suspendDefaultAction(); + inputProcessor.isAvailable().thenRun(suspendedDefaultAction::resume); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index e9cbc28378..23b1a46258 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -109,6 +109,7 @@ import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorStateContext; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; +import org.apache.flink.streaming.runtime.io.InputStatus; import org.apache.flink.streaming.runtime.io.StreamInputProcessor; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; @@ -1150,19 +1151,14 @@ public class StreamTaskTest extends TestLogger { } @Override - public boolean processInput() throws Exception { - return false; + public InputStatus processInput() throws Exception { + return isFinished ? InputStatus.END_OF_INPUT : InputStatus.NOTHING_AVAILABLE; } @Override public void close() throws IOException { } - @Override - public boolean isFinished() { - return isFinished; - } - @Override public CompletableFuture isAvailable() { return AVAILABLE; -- Gitee From 9d9d9e88a84ea45e205d3101991e13cca49f5bc6 Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Mon, 23 Sep 2019 15:36:21 +0800 Subject: [PATCH 033/268] [FLINK-14070] Use TimeUtils to parse duration configs (akka unrelated parts) --- ...e_rate_restart_strategy_configuration.html | 4 +- ..._delay_restart_strategy_configuration.html | 2 +- .../configuration/RestartStrategyOptions.java | 15 +++--- .../restart/FailureRateRestartStrategy.java | 8 +-- .../restart/FixedDelayRestartStrategy.java | 9 ++-- .../TaskManagerConfiguration.java | 52 +++++++------------ 6 files changed, 35 insertions(+), 55 deletions(-) diff --git a/docs/_includes/generated/failure_rate_restart_strategy_configuration.html b/docs/_includes/generated/failure_rate_restart_strategy_configuration.html index 4c41f4a941..589dce9855 100644 --- a/docs/_includes/generated/failure_rate_restart_strategy_configuration.html +++ b/docs/_includes/generated/failure_rate_restart_strategy_configuration.html @@ -10,12 +10,12 @@
restart-strategy.failure-rate.delay
"1 s" - Delay between two consecutive restart attempts if `restart-strategy` has been set to `failure-rate`. It can be specified using Scala's `FiniteDuration` notation: "1 min", "20 s" + Delay between two consecutive restart attempts if `restart-strategy` has been set to `failure-rate`. It can be specified using notation: "1 min", "20 s"
restart-strategy.failure-rate.failure-rate-interval
"1 min" - Time interval for measuring failure rate if `restart-strategy` has been set to `failure-rate`. It can be specified using Scala's `FiniteDuration` notation: "1 min", "20 s" + Time interval for measuring failure rate if `restart-strategy` has been set to `failure-rate`. It can be specified using notation: "1 min", "20 s"
restart-strategy.failure-rate.max-failures-per-interval
diff --git a/docs/_includes/generated/fixed_delay_restart_strategy_configuration.html b/docs/_includes/generated/fixed_delay_restart_strategy_configuration.html index f9c470999a..9e11b7fa20 100644 --- a/docs/_includes/generated/fixed_delay_restart_strategy_configuration.html +++ b/docs/_includes/generated/fixed_delay_restart_strategy_configuration.html @@ -15,7 +15,7 @@
restart-strategy.fixed-delay.delay
"1 s" - Delay between two consecutive restart attempts if `restart-strategy` has been set to `fixed-delay`. Delaying the retries can be helpful when the program interacts with external systems where for example connections or pending transactions should reach a timeout before re-execution is attempted. It can be specified using Scala's `FiniteDuration` notation: "1 min", "20 s" + Delay between two consecutive restart attempts if `restart-strategy` has been set to `fixed-delay`. Delaying the retries can be helpful when the program interacts with external systems where for example connections or pending transactions should reach a timeout before re-execution is attempted. It can be specified using notation: "1 min", "20 s" diff --git a/flink-core/src/main/java/org/apache/flink/configuration/RestartStrategyOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/RestartStrategyOptions.java index 6176891aee..a037fd66aa 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/RestartStrategyOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/RestartStrategyOptions.java @@ -92,10 +92,9 @@ public class RestartStrategyOptions { "Delay between two consecutive restart attempts if %s has been set to %s. " + "Delaying the retries can be helpful when the program interacts with external systems where " + "for example connections or pending transactions should reach a timeout before re-execution " + - "is attempted. It can be specified using Scala's %s notation: \"1 min\", \"20 s\"", + "is attempted. It can be specified using notation: \"1 min\", \"20 s\"", code(RESTART_STRATEGY.key()), - code("fixed-delay"), - code("FiniteDuration")) + code("fixed-delay")) .build()); public static final ConfigOption RESTART_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL = ConfigOptions @@ -116,10 +115,9 @@ public class RestartStrategyOptions { Description.builder() .text( "Time interval for measuring failure rate if %s has been set to %s. " + - "It can be specified using Scala's %s notation: \"1 min\", \"20 s\"", + "It can be specified using notation: \"1 min\", \"20 s\"", code(RESTART_STRATEGY.key()), - code("failure-rate"), - code("FiniteDuration")) + code("failure-rate")) .build()); public static final ConfigOption RESTART_STRATEGY_FAILURE_RATE_DELAY = ConfigOptions @@ -129,10 +127,9 @@ public class RestartStrategyOptions { Description.builder() .text( "Delay between two consecutive restart attempts if %s has been set to %s. " + - "It can be specified using Scala's %s notation: \"1 min\", \"20 s\"", + "It can be specified using notation: \"1 min\", \"20 s\"", code(RESTART_STRATEGY.key()), - code("failure-rate"), - code("FiniteDuration")) + code("failure-rate")) .build()); private RestartStrategyOptions() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FailureRateRestartStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FailureRateRestartStrategy.java index 291ba5eda3..69c6350baf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FailureRateRestartStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FailureRateRestartStrategy.java @@ -25,12 +25,12 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TimeUtils; +import java.time.Duration; import java.util.ArrayDeque; import java.util.concurrent.CompletableFuture; -import scala.concurrent.duration.Duration; - /** * Restart strategy which tries to restart the given {@link ExecutionGraph} when failure rate exceeded * with a fixed time delay in between. @@ -94,8 +94,8 @@ public class FailureRateRestartStrategy implements RestartStrategy { String failuresIntervalString = configuration.getString(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL); String delayString = configuration.getString(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_DELAY); - Duration failuresInterval = Duration.apply(failuresIntervalString); - Duration delay = Duration.apply(delayString); + Duration failuresInterval = TimeUtils.parseDuration(failuresIntervalString); + Duration delay = TimeUtils.parseDuration(delayString); return new FailureRateRestartStrategyFactory(maxFailuresPerInterval, Time.milliseconds(failuresInterval.toMillis()), Time.milliseconds(delay.toMillis())); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java index 01e305639b..95b84cb49f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java @@ -25,11 +25,10 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TimeUtils; import java.util.concurrent.CompletableFuture; -import scala.concurrent.duration.Duration; - /** * Restart strategy which tries to restart the given {@link ExecutionGraph} a fixed number of times * with a fixed time delay in between. @@ -82,11 +81,11 @@ public class FixedDelayRestartStrategy implements RestartStrategy { long delay; try { - delay = Duration.apply(delayString).toMillis(); - } catch (NumberFormatException nfe) { + delay = TimeUtils.parseDuration(delayString).toMillis(); + } catch (IllegalArgumentException ex) { throw new Exception("Invalid config value for " + RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY.key() + ": " + delayString + - ". Value must be a valid duration (such as '100 milli' or '10 s')"); + ". Value must be a valid duration (such as '100 milli' or '10 s')", ex); } return new FixedDelayRestartStrategyFactory(maxAttempts, delay); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java index b73d4971b1..67d7870b71 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java @@ -31,13 +31,14 @@ import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders import org.apache.flink.runtime.registration.RetryingRegistrationConfiguration; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TimeUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import scala.concurrent.duration.Duration; +import java.time.Duration; /** * Configuration object for {@link TaskExecutor}. @@ -196,56 +197,39 @@ public class TaskManagerConfiguration implements TaskManagerRuntimeInfo { LOG.info("Messages have a max timeout of " + timeout); - final Time finiteRegistrationDuration; - + Time finiteRegistrationDuration; try { - Duration maxRegistrationDuration = Duration.create(configuration.getString(TaskManagerOptions.REGISTRATION_TIMEOUT)); - if (maxRegistrationDuration.isFinite()) { - finiteRegistrationDuration = Time.milliseconds(maxRegistrationDuration.toMillis()); - } else { - finiteRegistrationDuration = null; - } - } catch (NumberFormatException e) { - throw new IllegalArgumentException("Invalid format for parameter " + - TaskManagerOptions.REGISTRATION_TIMEOUT.key(), e); + Duration maxRegistrationDuration = TimeUtils.parseDuration(configuration.getString(TaskManagerOptions.REGISTRATION_TIMEOUT)); + finiteRegistrationDuration = Time.milliseconds(maxRegistrationDuration.toMillis()); + } catch (IllegalArgumentException e) { + LOG.warn("Invalid format for parameter {}. Set the timeout to be infinite.", + TaskManagerOptions.REGISTRATION_TIMEOUT.key()); + finiteRegistrationDuration = null; } final Time initialRegistrationPause; try { - Duration pause = Duration.create(configuration.getString(TaskManagerOptions.INITIAL_REGISTRATION_BACKOFF)); - if (pause.isFinite()) { - initialRegistrationPause = Time.milliseconds(pause.toMillis()); - } else { - throw new IllegalArgumentException("The initial registration pause must be finite: " + pause); - } - } catch (NumberFormatException e) { + Duration pause = TimeUtils.parseDuration(configuration.getString(TaskManagerOptions.INITIAL_REGISTRATION_BACKOFF)); + initialRegistrationPause = Time.milliseconds(pause.toMillis()); + } catch (IllegalArgumentException e) { throw new IllegalArgumentException("Invalid format for parameter " + TaskManagerOptions.INITIAL_REGISTRATION_BACKOFF.key(), e); } final Time maxRegistrationPause; try { - Duration pause = Duration.create(configuration.getString( - TaskManagerOptions.REGISTRATION_MAX_BACKOFF)); - if (pause.isFinite()) { - maxRegistrationPause = Time.milliseconds(pause.toMillis()); - } else { - throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause); - } - } catch (NumberFormatException e) { + Duration pause = TimeUtils.parseDuration(configuration.getString(TaskManagerOptions.REGISTRATION_MAX_BACKOFF)); + maxRegistrationPause = Time.milliseconds(pause.toMillis()); + } catch (IllegalArgumentException e) { throw new IllegalArgumentException("Invalid format for parameter " + TaskManagerOptions.INITIAL_REGISTRATION_BACKOFF.key(), e); } final Time refusedRegistrationPause; try { - Duration pause = Duration.create(configuration.getString(TaskManagerOptions.REFUSED_REGISTRATION_BACKOFF)); - if (pause.isFinite()) { - refusedRegistrationPause = Time.milliseconds(pause.toMillis()); - } else { - throw new IllegalArgumentException("The refused registration pause must be finite: " + pause); - } - } catch (NumberFormatException e) { + Duration pause = TimeUtils.parseDuration(configuration.getString(TaskManagerOptions.REFUSED_REGISTRATION_BACKOFF)); + refusedRegistrationPause = Time.milliseconds(pause.toMillis()); + } catch (IllegalArgumentException e) { throw new IllegalArgumentException("Invalid format for parameter " + TaskManagerOptions.INITIAL_REGISTRATION_BACKOFF.key(), e); } -- Gitee From 949776bb8a8c326368bf7773becb8200610d5379 Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Mon, 23 Sep 2019 16:48:47 +0800 Subject: [PATCH 034/268] [FLINK-14070] Use TimeUtils to parse duration configs (akka related parts) This closes #9745. --- .../apache/flink/client/cli/CliFrontend.java | 5 +- .../flink/client/program/ClusterClient.java | 5 +- .../java/org/apache/flink/util/TimeUtils.java | 8 +++ .../org/apache/flink/util/TimeUtilsTest.java | 10 ++++ .../jobmaster/JobManagerSharedServices.java | 5 +- .../minicluster/MiniClusterConfiguration.java | 5 +- ...ceManagerRuntimeServicesConfiguration.java | 8 +-- .../slotmanager/SlotManagerConfiguration.java | 9 ++-- .../rpc/akka/AkkaRpcServiceConfiguration.java | 5 +- .../TaskManagerConfiguration.java | 3 +- .../apache/flink/runtime/akka/AkkaUtils.scala | 53 ++++++------------- .../flink/runtime/akka/AkkaUtilsTest.scala | 10 ++++ 12 files changed, 59 insertions(+), 67 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java index d1a635f20f..c4d5fa9b3e 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java @@ -73,6 +73,7 @@ import java.lang.reflect.UndeclaredThrowableException; import java.net.InetSocketAddress; import java.net.URL; import java.text.SimpleDateFormat; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -84,8 +85,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import scala.concurrent.duration.FiniteDuration; - /** * Implementation of a simple command line frontend for executing programs. */ @@ -113,7 +112,7 @@ public class CliFrontend { private final Options customCommandLineOptions; - private final FiniteDuration clientTimeout; + private final Duration clientTimeout; private final int defaultParallelism; diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index f90435332e..a61bae220a 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -52,13 +52,12 @@ import javax.annotation.Nullable; import java.net.URISyntaxException; import java.net.URL; +import java.time.Duration; import java.util.Collection; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; -import scala.concurrent.duration.FiniteDuration; - /** * Encapsulates the functionality necessary to submit a program to a remote cluster. * @@ -75,7 +74,7 @@ public abstract class ClusterClient implements AutoCloseable { protected final Configuration flinkConfig; /** Timeout for futures. */ - protected final FiniteDuration timeout; + protected final Duration timeout; /** * For interactive invocations, the job results are only available after the ContextEnvironment has * been run inside the user JAR. We pass the Client to every instance of the ContextEnvironment diff --git a/flink-core/src/main/java/org/apache/flink/util/TimeUtils.java b/flink-core/src/main/java/org/apache/flink/util/TimeUtils.java index a1e1b5dc36..1682ea896f 100644 --- a/flink-core/src/main/java/org/apache/flink/util/TimeUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/TimeUtils.java @@ -108,6 +108,14 @@ public class TimeUtils { return labelToUnit; } + /** + * @param duration to convert to string + * @return duration string in millis + */ + public static String getStringInMillis(final Duration duration) { + return duration.toMillis() + TimeUnit.MILLISECONDS.labels.get(0); + } + /** * Enum which defines time unit, mostly used to parse value from configuration file. */ diff --git a/flink-core/src/test/java/org/apache/flink/util/TimeUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/TimeUtilsTest.java index 49b2e86d01..6953c18a00 100644 --- a/flink-core/src/test/java/org/apache/flink/util/TimeUtilsTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/TimeUtilsTest.java @@ -20,6 +20,9 @@ package org.apache.flink.util; import org.junit.Test; +import java.time.Duration; +import java.time.temporal.ChronoUnit; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -166,4 +169,11 @@ public class TimeUtilsTest { public void testParseDurationNumberOverflow() { TimeUtils.parseDuration("100000000000000000000000000000000 ms"); } + + @Test + public void testGetStringInMillis() { + assertEquals("4567ms", TimeUtils.getStringInMillis(Duration.ofMillis(4567L))); + assertEquals("4567000ms", TimeUtils.getStringInMillis(Duration.ofSeconds(4567L))); + assertEquals("4ms", TimeUtils.getStringInMillis(Duration.of(4567L, ChronoUnit.MICROS))); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerSharedServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerSharedServices.java index 6c6696fed3..72e870cfea 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerSharedServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerSharedServices.java @@ -39,12 +39,11 @@ import org.apache.flink.util.ExceptionUtils; import javax.annotation.Nonnull; +import java.time.Duration; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import scala.concurrent.duration.FiniteDuration; - import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -152,7 +151,7 @@ public class JobManagerSharedServices { FlinkUserCodeClassLoaders.ResolveOrder.fromString(classLoaderResolveOrder), alwaysParentFirstLoaderPatterns); - final FiniteDuration timeout; + final Duration timeout; try { timeout = AkkaUtils.getTimeout(config); } catch (NumberFormatException e) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java index 20eee1bed9..6518aa0fb2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java @@ -30,8 +30,6 @@ import org.apache.flink.util.StringUtils; import javax.annotation.Nullable; -import scala.concurrent.duration.FiniteDuration; - import static org.apache.flink.runtime.minicluster.RpcServiceSharing.SHARED; /** @@ -104,8 +102,7 @@ public class MiniClusterConfiguration { } public Time getRpcTimeout() { - FiniteDuration duration = AkkaUtils.getTimeout(configuration); - return Time.of(duration.length(), duration.unit()); + return AkkaUtils.getTimeoutAsTime(configuration); } public UnmodifiableConfiguration getConfiguration() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServicesConfiguration.java index d4bc2d5dea..8331b61266 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServicesConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServicesConfiguration.java @@ -21,10 +21,10 @@ package org.apache.flink.runtime.resourcemanager; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ResourceManagerOptions; -import org.apache.flink.util.ConfigurationException; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerConfiguration; +import org.apache.flink.util.ConfigurationException; import org.apache.flink.util.Preconditions; -import scala.concurrent.duration.Duration; +import org.apache.flink.util.TimeUtils; /** * Configuration class for the {@link ResourceManagerRuntimeServices} class. @@ -56,8 +56,8 @@ public class ResourceManagerRuntimeServicesConfiguration { final Time jobTimeout; try { - jobTimeout = Time.milliseconds(Duration.apply(strJobTimeout).toMillis()); - } catch (NumberFormatException e) { + jobTimeout = Time.milliseconds(TimeUtils.parseDuration(strJobTimeout).toMillis()); + } catch (IllegalArgumentException e) { throw new ConfigurationException("Could not parse the resource manager's job timeout " + "value " + ResourceManagerOptions.JOB_TIMEOUT + '.', e); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java index b860c9bccb..c50a9a2882 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerConfiguration.java @@ -23,14 +23,13 @@ import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.ResourceManagerOptions; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.util.ConfigurationException; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.concurrent.duration.Duration; - /** * Configuration for the {@link SlotManager}. */ @@ -72,12 +71,10 @@ public class SlotManagerConfiguration { } public static SlotManagerConfiguration fromConfiguration(Configuration configuration) throws ConfigurationException { - final String strTimeout = configuration.getString(AkkaOptions.ASK_TIMEOUT); final Time rpcTimeout; - try { - rpcTimeout = Time.milliseconds(Duration.apply(strTimeout).toMillis()); - } catch (NumberFormatException e) { + rpcTimeout = AkkaUtils.getTimeoutAsTime(configuration); + } catch (IllegalArgumentException e) { throw new ConfigurationException("Could not parse the resource manager's timeout " + "value " + AkkaOptions.ASK_TIMEOUT + '.', e); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceConfiguration.java index 41f4c71ed1..0c478a9cbc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceConfiguration.java @@ -23,8 +23,6 @@ import org.apache.flink.runtime.akka.AkkaUtils; import javax.annotation.Nonnull; -import scala.concurrent.duration.FiniteDuration; - import static org.apache.flink.util.Preconditions.checkArgument; /** @@ -62,8 +60,7 @@ public class AkkaRpcServiceConfiguration { } public static AkkaRpcServiceConfiguration fromConfiguration(Configuration configuration) { - final FiniteDuration duration = AkkaUtils.getTimeout(configuration); - final Time timeout = Time.of(duration.length(), duration.unit()); + final Time timeout = AkkaUtils.getTimeoutAsTime(configuration); final long maximumFramesize = AkkaRpcServiceUtils.extractMaximumFramesize(configuration); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java index 67d7870b71..af71bbdee4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java @@ -186,9 +186,8 @@ public class TaskManagerConfiguration implements TaskManagerRuntimeInfo { final String[] tmpDirPaths = ConfigurationUtils.parseTempDirectories(configuration); final Time timeout; - try { - timeout = Time.milliseconds(AkkaUtils.getTimeout(configuration).toMillis()); + timeout = AkkaUtils.getTimeoutAsTime(configuration); } catch (Exception e) { throw new IllegalArgumentException( "Invalid format for '" + AkkaOptions.ASK_TIMEOUT.key() + diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index 531f04e732..4b4d634b7d 100755 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -20,7 +20,8 @@ package org.apache.flink.runtime.akka import java.io.IOException import java.net._ -import java.util.concurrent.{Callable, CompletableFuture, TimeUnit} +import java.time +import java.util.concurrent.{Callable, CompletableFuture} import akka.actor._ import akka.pattern.{ask => akkaAsk} @@ -31,6 +32,7 @@ import org.apache.flink.runtime.clusterframework.BootstrapTools.{FixedThreadPool import org.apache.flink.runtime.concurrent.FutureUtils import org.apache.flink.runtime.net.SSLUtils import org.apache.flink.util.NetUtils +import org.apache.flink.util.TimeUtils import org.apache.flink.util.function.FunctionUtils import org.jboss.netty.channel.ChannelException import org.jboss.netty.logging.{InternalLoggerFactory, Slf4JLoggerFactory} @@ -49,8 +51,6 @@ import scala.language.postfixOps object AkkaUtils { val LOG: Logger = LoggerFactory.getLogger(AkkaUtils.getClass) - val INF_TIMEOUT: FiniteDuration = 21474835 seconds - val FLINK_ACTOR_SYSTEM_NAME = "flink" def getFlinkActorSystemName = { @@ -367,9 +367,9 @@ object AkkaUtils { pauseValue: String, intervalParamName: String, intervalValue: String): Unit = { - if (Duration.apply(pauseValue).lteq(Duration.apply(intervalValue))) { + if (TimeUtils.parseDuration(pauseValue).compareTo(TimeUtils.parseDuration(intervalValue)) <= 0) { throw new IllegalConfigurationException( - "%s [%s] must greater then %s [%s]", + "%s [%s] must greater than %s [%s]", pauseParamName, pauseValue, intervalParamName, @@ -397,11 +397,11 @@ object AkkaUtils { val normalizedExternalHostname = NetUtils.unresolvedHostToNormalizedString(externalHostname) - val akkaAskTimeout = Duration(configuration.getString(AkkaOptions.ASK_TIMEOUT)) + val akkaAskTimeout = getTimeout(configuration) val startupTimeout = configuration.getString( AkkaOptions.STARTUP_TIMEOUT, - (akkaAskTimeout * 10).toString) + TimeUtils.getStringInMillis(akkaAskTimeout.multipliedBy(10L))) val transportHeartbeatInterval = configuration.getString( AkkaOptions.TRANSPORT_HEARTBEAT_INTERVAL) @@ -728,15 +728,13 @@ object AkkaUtils { } } - def getTimeout(config: Configuration): FiniteDuration = { - val duration = Duration(config.getString(AkkaOptions.ASK_TIMEOUT)) - - new FiniteDuration(duration.toMillis, TimeUnit.MILLISECONDS) + def getTimeout(config: Configuration): time.Duration = { + TimeUtils.parseDuration(config.getString(AkkaOptions.ASK_TIMEOUT)) } def getTimeoutAsTime(config: Configuration): Time = { try { - val duration = Duration(config.getString(AkkaOptions.ASK_TIMEOUT)) + val duration = getTimeout(config) Time.milliseconds(duration.toMillis) } catch { @@ -746,38 +744,17 @@ object AkkaUtils { } def getDefaultTimeout: Time = { - val duration = Duration(AkkaOptions.ASK_TIMEOUT.defaultValue()) + val duration = TimeUtils.parseDuration(AkkaOptions.ASK_TIMEOUT.defaultValue()) Time.milliseconds(duration.toMillis) } - def getDefaultTimeoutAsFiniteDuration: FiniteDuration = { - val timeout = getDefaultTimeout - - new FiniteDuration(timeout.toMilliseconds, TimeUnit.MILLISECONDS) - } - - def getLookupTimeout(config: Configuration): FiniteDuration = { - val duration = Duration(config.getString(AkkaOptions.LOOKUP_TIMEOUT)) - - new FiniteDuration(duration.toMillis, TimeUnit.MILLISECONDS) - } - - def getDefaultLookupTimeout: FiniteDuration = { - val duration = Duration(AkkaOptions.LOOKUP_TIMEOUT.defaultValue()) - new FiniteDuration(duration.toMillis, TimeUnit.MILLISECONDS) - } - - def getClientTimeout(config: Configuration): FiniteDuration = { - val duration = Duration(config.getString(AkkaOptions.CLIENT_TIMEOUT)) - - new FiniteDuration(duration.toMillis, TimeUnit.MILLISECONDS) + def getLookupTimeout(config: Configuration): time.Duration = { + TimeUtils.parseDuration(config.getString(AkkaOptions.LOOKUP_TIMEOUT)) } - def getDefaultClientTimeout: FiniteDuration = { - val duration = Duration(AkkaOptions.CLIENT_TIMEOUT.defaultValue()) - - new FiniteDuration(duration.toMillis, TimeUnit.MILLISECONDS) + def getClientTimeout(config: Configuration): time.Duration = { + TimeUtils.parseDuration(config.getString(AkkaOptions.CLIENT_TIMEOUT)) } /** Returns the address of the given [[ActorSystem]]. The [[Address]] object contains diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/akka/AkkaUtilsTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/akka/AkkaUtilsTest.scala index 3e20c44f3e..bc7b709655 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/akka/AkkaUtilsTest.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/akka/AkkaUtilsTest.scala @@ -207,4 +207,14 @@ class AkkaUtilsTest akkaConfig.getString("akka.remote.netty.tcp.hostname") should equal(NetUtils.unresolvedHostToNormalizedString(ipv6AddressString)) } + + test("getAkkaConfig should set startup timeout to be 10 times of ask timeout by default") { + val configuration = new Configuration() + configuration.setString(AkkaOptions.ASK_TIMEOUT.key(), "100ms") + + val akkaConfig = AkkaUtils.getAkkaConfig(configuration, Some(("localhost", 31337))) + + akkaConfig.getString("akka.remote.startup-timeout") should + equal("1000ms") + } } -- Gitee From 8d29cd4ee782f857be56f89cffb4a24a34cce45b Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Wed, 25 Sep 2019 02:13:14 +0800 Subject: [PATCH 035/268] [hotfix] Convert Akka duration configs to valid Akka duration format --- .../apache/flink/runtime/akka/AkkaUtils.scala | 33 +++++++++++-------- 1 file changed, 20 insertions(+), 13 deletions(-) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index 4b4d634b7d..7070df46e7 100755 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -364,10 +364,10 @@ object AkkaUtils { } private def validateHeartbeat(pauseParamName: String, - pauseValue: String, + pauseValue: time.Duration, intervalParamName: String, - intervalValue: String): Unit = { - if (TimeUtils.parseDuration(pauseValue).compareTo(TimeUtils.parseDuration(intervalValue)) <= 0) { + intervalValue: time.Duration): Unit = { + if (pauseValue.compareTo(intervalValue) <= 0) { throw new IllegalConfigurationException( "%s [%s] must greater than %s [%s]", pauseParamName, @@ -399,25 +399,32 @@ object AkkaUtils { val akkaAskTimeout = getTimeout(configuration) - val startupTimeout = configuration.getString( - AkkaOptions.STARTUP_TIMEOUT, - TimeUtils.getStringInMillis(akkaAskTimeout.multipliedBy(10L))) + val startupTimeout = TimeUtils.getStringInMillis( + TimeUtils.parseDuration( + configuration.getString( + AkkaOptions.STARTUP_TIMEOUT, + TimeUtils.getStringInMillis(akkaAskTimeout.multipliedBy(10L))))) - val transportHeartbeatInterval = configuration.getString( - AkkaOptions.TRANSPORT_HEARTBEAT_INTERVAL) + val transportHeartbeatIntervalDuration = TimeUtils.parseDuration( + configuration.getString(AkkaOptions.TRANSPORT_HEARTBEAT_INTERVAL)) - val transportHeartbeatPause = configuration.getString( - AkkaOptions.TRANSPORT_HEARTBEAT_PAUSE) + val transportHeartbeatPauseDuration = TimeUtils.parseDuration( + configuration.getString(AkkaOptions.TRANSPORT_HEARTBEAT_PAUSE)) validateHeartbeat( AkkaOptions.TRANSPORT_HEARTBEAT_PAUSE.key(), - transportHeartbeatPause, + transportHeartbeatPauseDuration, AkkaOptions.TRANSPORT_HEARTBEAT_INTERVAL.key(), - transportHeartbeatInterval) + transportHeartbeatIntervalDuration) + + val transportHeartbeatInterval = TimeUtils.getStringInMillis(transportHeartbeatIntervalDuration) + + val transportHeartbeatPause = TimeUtils.getStringInMillis(transportHeartbeatPauseDuration) val transportThreshold = configuration.getDouble(AkkaOptions.TRANSPORT_THRESHOLD) - val akkaTCPTimeout = configuration.getString(AkkaOptions.TCP_TIMEOUT) + val akkaTCPTimeout = TimeUtils.getStringInMillis( + TimeUtils.parseDuration(configuration.getString(AkkaOptions.TCP_TIMEOUT))) val akkaFramesize = configuration.getString(AkkaOptions.FRAMESIZE) -- Gitee From 8b85715cfb9d9927c7d2fb82593be0bcaf084502 Mon Sep 17 00:00:00 2001 From: tison Date: Mon, 23 Sep 2019 14:26:21 +0800 Subject: [PATCH 036/268] [hotfix][coordination] Fix MiniCluster#closeAsync to correctly close all components and services --- .../apache/flink/runtime/minicluster/MiniCluster.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) 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 2023f7bc22..41e265f02d 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 @@ -97,7 +97,6 @@ import java.util.List; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; -import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; @@ -430,15 +429,15 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { componentsTerminationFuture, this::closeMetricSystem); - // shut down the RpcServices - final CompletableFuture rpcServicesTerminationFuture = metricSystemTerminationFuture - .thenCompose((Void ignored) -> terminateRpcServices()); + final CompletableFuture rpcServicesTerminationFuture = FutureUtils.composeAfterwards( + metricSystemTerminationFuture, + this::terminateRpcServices); final CompletableFuture remainingServicesTerminationFuture = FutureUtils.runAfterwards( rpcServicesTerminationFuture, this::terminateMiniClusterServices); - final CompletableFuture executorsTerminationFuture = FutureUtils.runAfterwards( + final CompletableFuture executorsTerminationFuture = FutureUtils.composeAfterwards( remainingServicesTerminationFuture, () -> terminateExecutors(shutdownTimeoutMillis)); @@ -832,7 +831,7 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { } @Nonnull - private CompletionStage terminateRpcServices() { + private CompletableFuture terminateRpcServices() { synchronized (lock) { final int numRpcServices = 1 + rpcServices.size(); -- Gitee From 26b31565404637dd451e04cb4f22cb777b0b8c10 Mon Sep 17 00:00:00 2001 From: "kevin.cyj" Date: Sun, 22 Sep 2019 15:57:40 +0800 Subject: [PATCH 037/268] [FLINK-14180] Enable config of maximum capacity of FileArchivedExecutionGraphStore. This closes #9753. --- .../generated/job_manager_configuration.html | 5 ++ .../configuration/JobManagerOptions.java | 8 +++ .../FileArchivedExecutionGraphStore.java | 2 + .../entrypoint/SessionClusterEntrypoint.java | 2 + .../FileArchivedExecutionGraphStoreTest.java | 52 ++++++++++++++++++- 5 files changed, 68 insertions(+), 1 deletion(-) diff --git a/docs/_includes/generated/job_manager_configuration.html b/docs/_includes/generated/job_manager_configuration.html index b4ae08e050..0a60f282ad 100644 --- a/docs/_includes/generated/job_manager_configuration.html +++ b/docs/_includes/generated/job_manager_configuration.html @@ -47,6 +47,11 @@ 3600 The time in seconds after which a completed job expires and is purged from the job store. + +
jobstore.max-capacity
+ 2147483647 + The max number of completed jobs that can be kept in the job store. +
slot.idle.timeout
50000 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 3643667664..a6d6f05daa 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 @@ -156,6 +156,14 @@ public class JobManagerOptions { .defaultValue(60L * 60L) .withDescription("The time in seconds after which a completed job expires and is purged from the job store."); + /** + * The max number of completed jobs that can be kept in the job store. + */ + public static final ConfigOption JOB_STORE_MAX_CAPACITY = + key("jobstore.max-capacity") + .defaultValue(Integer.MAX_VALUE) + .withDescription("The max number of completed jobs that can be kept in the job store."); + /** * The timeout in milliseconds for requesting a slot from Slot Pool. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/FileArchivedExecutionGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/FileArchivedExecutionGraphStore.java index 6526072414..bd607e3fee 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/FileArchivedExecutionGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/FileArchivedExecutionGraphStore.java @@ -83,6 +83,7 @@ public class FileArchivedExecutionGraphStore implements ArchivedExecutionGraphSt public FileArchivedExecutionGraphStore( File rootDir, Time expirationTime, + int maximumCapacity, long maximumCacheSizeBytes, ScheduledExecutor scheduledExecutor, Ticker ticker) throws IOException { @@ -102,6 +103,7 @@ public class FileArchivedExecutionGraphStore implements ArchivedExecutionGraphSt "The storage directory must exist and be a directory."); this.jobDetailsCache = CacheBuilder.newBuilder() .expireAfterWrite(expirationTime.toMilliseconds(), TimeUnit.MILLISECONDS) + .maximumSize(maximumCapacity) .removalListener( (RemovalListener) notification -> deleteExecutionGraphFile(notification.getKey())) .ticker(ticker) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java index 1fb693cf83..42daeae013 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java @@ -47,11 +47,13 @@ public abstract class SessionClusterEntrypoint extends ClusterEntrypoint { final File tmpDir = new File(ConfigurationUtils.parseTempDirectories(configuration)[0]); final Time expirationTime = Time.seconds(configuration.getLong(JobManagerOptions.JOB_STORE_EXPIRATION_TIME)); + final int maximumCapacity = configuration.getInteger(JobManagerOptions.JOB_STORE_MAX_CAPACITY); final long maximumCacheSizeBytes = configuration.getLong(JobManagerOptions.JOB_STORE_CACHE_SIZE); return new FileArchivedExecutionGraphStore( tmpDir, expirationTime, + maximumCapacity, maximumCacheSizeBytes, scheduledExecutor, Ticker.systemTicker()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FileArchivedExecutionGraphStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FileArchivedExecutionGraphStoreTest.java index cc116dd771..b3b31ac757 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FileArchivedExecutionGraphStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FileArchivedExecutionGraphStoreTest.java @@ -54,7 +54,9 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; /** * Tests for the {@link FileArchivedExecutionGraphStore}. @@ -136,7 +138,7 @@ public class FileArchivedExecutionGraphStoreTest extends TestLogger { final int numberExecutionGraphs = 10; final Collection executionGraphs = generateTerminalExecutionGraphs(numberExecutionGraphs); - final Collection jobDetails = executionGraphs.stream().map(WebMonitorUtils::createDetailsForJob).collect(Collectors.toList()); + final Collection jobDetails = generateJobDetails(executionGraphs); final File rootDir = temporaryFolder.newFolder(); @@ -165,6 +167,7 @@ public class FileArchivedExecutionGraphStoreTest extends TestLogger { try (final FileArchivedExecutionGraphStore executionGraphStore = new FileArchivedExecutionGraphStore( rootDir, expirationTime, + Integer.MAX_VALUE, 10000L, scheduledExecutor, manualTicker)) { @@ -227,6 +230,7 @@ public class FileArchivedExecutionGraphStoreTest extends TestLogger { try (final FileArchivedExecutionGraphStore executionGraphStore = new FileArchivedExecutionGraphStore( rootDir, Time.hours(1L), + Integer.MAX_VALUE, 100L << 10, TestingUtils.defaultScheduledExecutor(), Ticker.systemTicker())) { @@ -259,6 +263,47 @@ public class FileArchivedExecutionGraphStoreTest extends TestLogger { } } + /** + * Tests that the size of {@link FileArchivedExecutionGraphStore} is no more than the configured max capacity + * and the old execution graphs will be purged if the total added number exceeds the max capacity. + */ + @Test + public void testMaximumCapacity() throws IOException { + final File rootDir = temporaryFolder.newFolder(); + + final int maxCapacity = 10; + final int numberExecutionGraphs = 10; + + final Collection oldExecutionGraphs = generateTerminalExecutionGraphs(numberExecutionGraphs); + final Collection newExecutionGraphs = generateTerminalExecutionGraphs(numberExecutionGraphs); + + final Collection jobDetails = generateJobDetails(newExecutionGraphs); + + try (final FileArchivedExecutionGraphStore executionGraphStore = new FileArchivedExecutionGraphStore( + rootDir, + Time.hours(1L), + maxCapacity, + 10000L, + TestingUtils.defaultScheduledExecutor(), + Ticker.systemTicker())) { + + for (ArchivedExecutionGraph executionGraph : oldExecutionGraphs) { + executionGraphStore.put(executionGraph); + // no more than the configured maximum capacity + assertTrue(executionGraphStore.size() <= maxCapacity); + } + + for (ArchivedExecutionGraph executionGraph : newExecutionGraphs) { + executionGraphStore.put(executionGraph); + // equals to the configured maximum capacity + assertEquals(maxCapacity, executionGraphStore.size()); + } + + // the older execution graphs are purged + assertThat(executionGraphStore.getAvailableJobDetails(), Matchers.containsInAnyOrder(jobDetails.toArray())); + } + } + private Collection generateTerminalExecutionGraphs(int number) { final Collection executionGraphs = new ArrayList<>(number); @@ -277,6 +322,7 @@ public class FileArchivedExecutionGraphStoreTest extends TestLogger { return new FileArchivedExecutionGraphStore( storageDirectory, Time.hours(1L), + Integer.MAX_VALUE, 10000L, TestingUtils.defaultScheduledExecutor(), Ticker.systemTicker()); @@ -318,4 +364,8 @@ public class FileArchivedExecutionGraphStoreTest extends TestLogger { private static Matcher matchesPartiallyWith(ArchivedExecutionGraph executionGraph) { return new PartialArchivedExecutionGraphMatcher(executionGraph); } + + private static Collection generateJobDetails(Collection executionGraphs) { + return executionGraphs.stream().map(WebMonitorUtils::createDetailsForJob).collect(Collectors.toList()); + } } -- Gitee From b6e32a317ec273ee3f3085728a02bc3922c22db6 Mon Sep 17 00:00:00 2001 From: GatsbyNewton Date: Tue, 24 Sep 2019 23:34:45 +0800 Subject: [PATCH 038/268] [FLINK-14031][examples] Add blink planner dependency and examples which use blink planner This closes #9761 --- flink-examples/flink-examples-table/pom.xml | 5 +++ .../table/examples/java/StreamSQLExample.java | 24 +++++++++++++- .../examples/scala/StreamSQLExample.scala | 33 ++++++++++++++++--- 3 files changed, 56 insertions(+), 6 deletions(-) diff --git a/flink-examples/flink-examples-table/pom.xml b/flink-examples/flink-examples-table/pom.xml index 7b45bdd986..cd58a57bb2 100644 --- a/flink-examples/flink-examples-table/pom.xml +++ b/flink-examples/flink-examples-table/pom.xml @@ -51,6 +51,11 @@ under the License. flink-table-planner_${scala.binary.version} ${project.version} + + org.apache.flink + flink-table-planner-blink_${scala.binary.version} + ${project.version} + org.apache.flink diff --git a/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/StreamSQLExample.java b/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/StreamSQLExample.java index d3f9e08628..e0af763982 100644 --- a/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/StreamSQLExample.java +++ b/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/StreamSQLExample.java @@ -18,16 +18,21 @@ package org.apache.flink.table.examples.java; +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.table.api.EnvironmentSettings; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import java.util.Arrays; +import java.util.Objects; /** * Simple example for demonstrating the use of SQL on a Stream Table in Java. * + *

Usage: StreamSQLExample --planner <blink|flink>
+ * *

This example shows how to: * - Convert DataStreams to Tables * - Register a Table under a name @@ -42,9 +47,26 @@ public class StreamSQLExample { public static void main(String[] args) throws Exception { + final ParameterTool params = ParameterTool.fromArgs(args); + String planner = params.has("planner") ? params.get("planner") : "flink"; + // set up execution environment StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + StreamTableEnvironment tEnv; + if (Objects.equals(planner, "blink")) { // use blink planner in streaming mode + EnvironmentSettings settings = EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build(); + tEnv = StreamTableEnvironment.create(env, settings); + } else if (Objects.equals(planner, "flink")) { // use flink planner in streaming mode + tEnv = StreamTableEnvironment.create(env); + } else { + System.err.println("The planner is incorrect. Please run 'StreamSQLExample --planner ', " + + "where planner (it is either flink or blink, and the default is flink) indicates whether the " + + "example uses flink planner or blink planner."); + return; + } DataStream orderA = env.fromCollection(Arrays.asList( new Order(1L, "beer", 3), diff --git a/flink-examples/flink-examples-table/src/main/scala/org/apache/flink/table/examples/scala/StreamSQLExample.scala b/flink-examples/flink-examples-table/src/main/scala/org/apache/flink/table/examples/scala/StreamSQLExample.scala index 2e922e02e3..79fa5e8179 100644 --- a/flink-examples/flink-examples-table/src/main/scala/org/apache/flink/table/examples/scala/StreamSQLExample.scala +++ b/flink-examples/flink-examples-table/src/main/scala/org/apache/flink/table/examples/scala/StreamSQLExample.scala @@ -17,14 +17,18 @@ */ package org.apache.flink.table.examples.scala +import org.apache.flink.api.java.utils.ParameterTool import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment} +import org.apache.flink.table.api.EnvironmentSettings import org.apache.flink.table.api.scala._ /** * Simple example for demonstrating the use of SQL on a Stream Table in Scala. * - * This example shows how to: + *

Usage: StreamSQLExample --planner <blink|flink>
+ * + *

This example shows how to: * - Convert DataStreams to Tables * - Register a Table under a name * - Run a StreamSQL query on the registered Table @@ -38,9 +42,25 @@ object StreamSQLExample { def main(args: Array[String]): Unit = { + val params = ParameterTool.fromArgs(args) + val planner = if (params.has("planner")) params.get("planner") else "flink" + // set up execution environment val env = StreamExecutionEnvironment.getExecutionEnvironment - val tEnv = StreamTableEnvironment.create(env) + val tEnv = if (planner == "blink") { // use blink planner in streaming mode + val settings = EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build() + StreamTableEnvironment.create(env, settings) + } else if (planner == "flink") { // use flink planner in streaming mode + StreamTableEnvironment.create(env) + } else { + System.err.println("The planner is incorrect. Please run 'StreamSQLExample --planner ', " + + "where planner (it is either flink or blink, and the default is flink) indicates whether the " + + "example uses flink planner or blink planner.") + return + } val orderA: DataStream[Order] = env.fromCollection(Seq( Order(1L, "beer", 3), @@ -53,14 +73,17 @@ object StreamSQLExample { Order(4L, "beer", 1))) // convert DataStream to Table - var tableA = tEnv.fromDataStream(orderA, 'user, 'product, 'amount) + val tableA = tEnv.fromDataStream(orderA, 'user, 'product, 'amount) // register DataStream as Table tEnv.registerDataStream("OrderB", orderB, 'user, 'product, 'amount) // union the two tables val result = tEnv.sqlQuery( - s"SELECT * FROM $tableA WHERE amount > 2 UNION ALL " + - "SELECT * FROM OrderB WHERE amount < 2") + s""" + |SELECT * FROM $tableA WHERE amount > 2 + |UNION ALL + |SELECT * FROM OrderB WHERE amount < 2 + """.stripMargin) result.toAppendStream[Order].print() -- Gitee From fe22f22ab5684c08a9bf10f81d3ecb7ea9269f7b Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Thu, 19 Sep 2019 22:47:12 +0800 Subject: [PATCH 039/268] [FLINK-14128][runtime,docs] Remove the description of restart strategy customization This closes #9722. --- .../generated/restart_strategy_configuration.html | 2 +- .../org/apache/flink/configuration/ConfigConstants.java | 4 +--- .../apache/flink/configuration/RestartStrategyOptions.java | 7 +------ 3 files changed, 3 insertions(+), 10 deletions(-) diff --git a/docs/_includes/generated/restart_strategy_configuration.html b/docs/_includes/generated/restart_strategy_configuration.html index d6c8d72bce..e17bbed899 100644 --- a/docs/_includes/generated/restart_strategy_configuration.html +++ b/docs/_includes/generated/restart_strategy_configuration.html @@ -10,7 +10,7 @@

restart-strategy
(none) - Defines the restart strategy to use in case of job failures.
Accepted values are:
  • `none`, `off`, `disable`: No restart strategy.
  • `fixeddelay`, `fixed-delay`: Fixed delay restart strategy. More details can be found here.
  • `failurerate`, `failure-rate`: Failure rate restart strategy. More details can be found here.
  • `org.foobar.MyRestartStrategyFactoryFactory`: Fully qualified name of `RestartStrategyFactory` factory which has has a method `RestartStrategyFactory createFactory(Configuration configuration)`.
If checkpointing is disabled, the default value is `none`. If checkpointing is enabled, the default value is `fixed-delay` with `Integer.MAX_VALUE` restart attempts and '`1 s`' delay. + Defines the restart strategy to use in case of job failures.
Accepted values are:
  • `none`, `off`, `disable`: No restart strategy.
  • `fixeddelay`, `fixed-delay`: Fixed delay restart strategy. More details can be found here.
  • `failurerate`, `failure-rate`: Failure rate restart strategy. More details can be found here.
If checkpointing is disabled, the default value is `none`. If checkpointing is enabled, the default value is `fixed-delay` with `Integer.MAX_VALUE` restart attempts and '`1 s`' delay. diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index 1a9fda51e6..9a4a9e0778 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -53,9 +53,7 @@ public final class ConfigConstants { /** * Defines the restart strategy to be used. It can be "off", "none", "disable" to be disabled or * it can be "fixeddelay", "fixed-delay" to use the FixedDelayRestartStrategy or it can - * be "failurerate", "failure-rate" to use FailureRateRestartStrategy. You can also - * specify a class name which implements the RestartStrategy interface and has a static - * create method which takes a Configuration object. + * be "failurerate", "failure-rate" to use FailureRateRestartStrategy. * * @deprecated use {@link RestartStrategyOptions#RESTART_STRATEGY} instead. */ diff --git a/flink-core/src/main/java/org/apache/flink/configuration/RestartStrategyOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/RestartStrategyOptions.java index a037fd66aa..8ff2581781 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/RestartStrategyOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/RestartStrategyOptions.java @@ -56,12 +56,7 @@ public class RestartStrategyOptions { "%s, %s: Failure rate restart strategy. More details can be found %s.", code("failurerate"), code("failure-rate"), - link("../dev/task_failure_recovery.html#failure-rate-restart-strategy", "here")), - text( - "%s: Fully qualified name of %s factory which has has a method %s.", - code("org.foobar.MyRestartStrategyFactoryFactory"), - code("RestartStrategyFactory"), - code("RestartStrategyFactory createFactory(Configuration configuration)")) + link("../dev/task_failure_recovery.html#failure-rate-restart-strategy", "here")) ) .text( "If checkpointing is disabled, the default value is %s. " + -- Gitee From 9015f3f596727e2fd565e90eb90fdd5106cc14fc Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Wed, 25 Sep 2019 16:37:57 +0800 Subject: [PATCH 040/268] [hotfix][runtime] Remove RestartBackoffTimeStrategyOptions and use RestartStrategyOptions instead --- ...t_backoff_time_strategy_configuration.html | 36 --------- .../RestartBackoffTimeStrategyOptions.java | 74 ------------------- ...FailureRateRestartBackoffTimeStrategy.java | 32 ++++++-- .../FixedDelayRestartBackoffTimeStrategy.java | 20 +++-- 4 files changed, 40 insertions(+), 122 deletions(-) delete mode 100644 docs/_includes/generated/restart_backoff_time_strategy_configuration.html delete mode 100644 flink-core/src/main/java/org/apache/flink/configuration/RestartBackoffTimeStrategyOptions.java diff --git a/docs/_includes/generated/restart_backoff_time_strategy_configuration.html b/docs/_includes/generated/restart_backoff_time_strategy_configuration.html deleted file mode 100644 index 33785e0c7f..0000000000 --- a/docs/_includes/generated/restart_backoff_time_strategy_configuration.html +++ /dev/null @@ -1,36 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
KeyDefaultDescription
restart-backoff-time-strategy.failure-rate.backoff-time
0Backoff time in milliseconds between two consecutive restart attempts.
restart-backoff-time-strategy.failure-rate.failure-rate-interval
60000Time interval in milliseconds for measuring failure rate.
restart-backoff-time-strategy.failure-rate.max-failures-per-interval
1Maximum number of failures in given time interval before failing a job.
restart-backoff-time-strategy.fixed-delay.attempts
2147483647Maximum number of attempts the fixed delay restart strategy will try before failing a job.
restart-backoff-time-strategy.fixed-delay.backoff-time
0Backoff time in milliseconds between two consecutive restart attempts.
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/RestartBackoffTimeStrategyOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/RestartBackoffTimeStrategyOptions.java deleted file mode 100644 index 7b56a336e3..0000000000 --- a/flink-core/src/main/java/org/apache/flink/configuration/RestartBackoffTimeStrategyOptions.java +++ /dev/null @@ -1,74 +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.configuration; - -import org.apache.flink.annotation.PublicEvolving; - -/** - * Configuration options for the RestartBackoffTimeStrategy. - */ -@PublicEvolving -public class RestartBackoffTimeStrategyOptions { - /** - * Maximum number of failures in given time interval {@link #RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL} - * before failing a job in FailureRateRestartBackoffTimeStrategy. - */ - @PublicEvolving - public static final ConfigOption RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL = ConfigOptions - .key("restart-backoff-time-strategy.failure-rate.max-failures-per-interval") - .defaultValue(1) - .withDescription("Maximum number of failures in given time interval before failing a job."); - - /** - * Time interval in which greater amount of failures than {@link #RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL} - * causes job fail in FailureRateRestartBackoffTimeStrategy. - */ - @PublicEvolving - public static final ConfigOption RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL = ConfigOptions - .key("restart-backoff-time-strategy.failure-rate.failure-rate-interval") - .defaultValue(60_000L) - .withDescription("Time interval in milliseconds for measuring failure rate."); - - /** - * Backoff time between two consecutive restart attempts in FailureRateRestartBackoffTimeStrategy. - */ - @PublicEvolving - public static final ConfigOption RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_BACKOFF_TIME = ConfigOptions - .key("restart-backoff-time-strategy.failure-rate.backoff-time") - .defaultValue(0L) - .withDescription("Backoff time in milliseconds between two consecutive restart attempts."); - - /** - * Maximum number of attempts the fixed delay restart strategy will try before failing a job. - */ - @PublicEvolving - public static final ConfigOption RESTART_BACKOFF_TIME_STRATEGY_FIXED_DELAY_ATTEMPTS = ConfigOptions - .key("restart-backoff-time-strategy.fixed-delay.attempts") - .defaultValue(Integer.MAX_VALUE) - .withDescription("Maximum number of attempts the fixed delay restart strategy will try before failing a job."); - - /** - * Backoff time between two consecutive restart attempts in FixedDelayRestartBackoffTimeStrategy. - */ - @PublicEvolving - public static final ConfigOption RESTART_BACKOFF_TIME_STRATEGY_FIXED_DELAY_BACKOFF_TIME = ConfigOptions - .key("restart-backoff-time-strategy.fixed-delay.backoff-time") - .defaultValue(0L) - .withDescription("Backoff time in milliseconds between two consecutive restart attempts."); -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureRateRestartBackoffTimeStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureRateRestartBackoffTimeStrategy.java index 8404ddaf1b..2d589112e8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureRateRestartBackoffTimeStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailureRateRestartBackoffTimeStrategy.java @@ -19,15 +19,14 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestartStrategyOptions; import org.apache.flink.runtime.util.clock.Clock; import org.apache.flink.runtime.util.clock.SystemClock; +import org.apache.flink.util.TimeUtils; import java.util.ArrayDeque; import java.util.Deque; -import static org.apache.flink.configuration.RestartBackoffTimeStrategyOptions.RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_BACKOFF_TIME; -import static org.apache.flink.configuration.RestartBackoffTimeStrategyOptions.RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL; -import static org.apache.flink.configuration.RestartBackoffTimeStrategyOptions.RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -110,10 +109,29 @@ public class FailureRateRestartBackoffTimeStrategy implements RestartBackoffTime } public static FailureRateRestartBackoffTimeStrategyFactory createFactory(final Configuration configuration) { - return new FailureRateRestartBackoffTimeStrategyFactory( - configuration.getInteger(RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL), - configuration.getLong(RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL), - configuration.getLong(RESTART_BACKOFF_TIME_STRATEGY_FAILURE_RATE_FAILURE_RATE_BACKOFF_TIME)); + int maxFailuresPerInterval = configuration.getInteger(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL); + String failuresIntervalString = configuration.getString(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL); + String delayString = configuration.getString(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_DELAY); + + long failuresInterval; + try { + failuresInterval = TimeUtils.parseDuration(failuresIntervalString).toMillis(); + } catch (IllegalArgumentException ex) { + throw new IllegalArgumentException("Invalid config value for " + + RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL.key() + ": " + failuresIntervalString + + ". Value must be a valid duration (such as '100 milli' or '10 s')", ex); + } + + long delay; + try { + delay = TimeUtils.parseDuration(delayString).toMillis(); + } catch (IllegalArgumentException ex) { + throw new IllegalArgumentException("Invalid config value for " + + RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_DELAY.key() + ": " + delayString + + ". Value must be a valid duration (such as '100 milli' or '10 s')", ex); + } + + return new FailureRateRestartBackoffTimeStrategyFactory(maxFailuresPerInterval, failuresInterval, delay); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FixedDelayRestartBackoffTimeStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FixedDelayRestartBackoffTimeStrategy.java index 64dc1bf928..3c5309e91e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FixedDelayRestartBackoffTimeStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FixedDelayRestartBackoffTimeStrategy.java @@ -19,9 +19,9 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestartStrategyOptions; +import org.apache.flink.util.TimeUtils; -import static org.apache.flink.configuration.RestartBackoffTimeStrategyOptions.RESTART_BACKOFF_TIME_STRATEGY_FIXED_DELAY_ATTEMPTS; -import static org.apache.flink.configuration.RestartBackoffTimeStrategyOptions.RESTART_BACKOFF_TIME_STRATEGY_FIXED_DELAY_BACKOFF_TIME; import static org.apache.flink.util.Preconditions.checkArgument; /** @@ -79,9 +79,19 @@ public class FixedDelayRestartBackoffTimeStrategy implements RestartBackoffTimeS } public static FixedDelayRestartBackoffTimeStrategyFactory createFactory(final Configuration configuration) { - return new FixedDelayRestartBackoffTimeStrategyFactory( - configuration.getInteger(RESTART_BACKOFF_TIME_STRATEGY_FIXED_DELAY_ATTEMPTS), - configuration.getLong(RESTART_BACKOFF_TIME_STRATEGY_FIXED_DELAY_BACKOFF_TIME)); + int maxAttempts = configuration.getInteger(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS); + String delayString = configuration.getString(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY); + + long delay; + try { + delay = TimeUtils.parseDuration(delayString).toMillis(); + } catch (IllegalArgumentException ex) { + throw new IllegalArgumentException("Invalid config value for " + + RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY.key() + ": " + delayString + + ". Value must be a valid duration (such as '100 milli' or '10 s')", ex); + } + + return new FixedDelayRestartBackoffTimeStrategyFactory(maxAttempts, delay); } /** -- Gitee From 9266f1440ff54c2cabc4645c36abaed5f3b53577 Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Wed, 25 Sep 2019 16:49:25 +0800 Subject: [PATCH 041/268] [FLINK-12709][runtime] Add NoRestartBackoffTimeStrategy which suppresses all task restarts --- .../flip1/NoRestartBackoffTimeStrategy.java | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/NoRestartBackoffTimeStrategy.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/NoRestartBackoffTimeStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/NoRestartBackoffTimeStrategy.java new file mode 100644 index 0000000000..fb625068b0 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/NoRestartBackoffTimeStrategy.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.executiongraph.failover.flip1; + +/** + * Restart strategy which does not restart tasks when tasks fail. + */ +public enum NoRestartBackoffTimeStrategy implements RestartBackoffTimeStrategy { + INSTANCE; + + @Override + public boolean canRestart() { + return false; + } + + @Override + public long getBackoffTime() { + return 0L; + } + + @Override + public void notifyFailure(final Throwable cause) { + // nothing to do + } + + @Override + public String toString() { + return "NoRestartBackoffTimeStrategy"; + } + + /** + * The factory for creating {@link NoRestartBackoffTimeStrategy}. + */ + public enum NoRestartBackoffTimeStrategyFactory implements Factory { + INSTANCE; + + @Override + public RestartBackoffTimeStrategy create() { + return NoRestartBackoffTimeStrategy.INSTANCE; + } + + @Override + public String toString() { + return "NoRestartBackoffTimeStrategyFactory"; + } + } +} -- Gitee From 13aca92d8c1f6a85ccbc8837958763b623f89ef1 Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Wed, 25 Sep 2019 16:51:14 +0800 Subject: [PATCH 042/268] [FLINK-12709][runtime] Implement RestartBackoffTimeStrategyFactoryLoader which also respects legacy restart strategy configs This closes #8912. --- .../FixedDelayRestartBackoffTimeStrategy.java | 4 + ...startBackoffTimeStrategyFactoryLoader.java | 139 ++++++++++++++ ...pointingEnabledRestartStrategyFactory.java | 6 +- ...tBackoffTimeStrategyFactoryLoaderTest.java | 179 ++++++++++++++++++ 4 files changed, 326 insertions(+), 2 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartBackoffTimeStrategyFactoryLoader.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartBackoffTimeStrategyFactoryLoaderTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FixedDelayRestartBackoffTimeStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FixedDelayRestartBackoffTimeStrategy.java index 3c5309e91e..ca786d3fb5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FixedDelayRestartBackoffTimeStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FixedDelayRestartBackoffTimeStrategy.java @@ -57,6 +57,10 @@ public class FixedDelayRestartBackoffTimeStrategy implements RestartBackoffTimeS return backoffTimeMS; } + public int getMaxNumberRestartAttempts() { + return maxNumberRestartAttempts; + } + @Override public void notifyFailure(Throwable cause) { currentRestartAttempt++; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartBackoffTimeStrategyFactoryLoader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartBackoffTimeStrategyFactoryLoader.java new file mode 100644 index 0000000000..6323f46acc --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartBackoffTimeStrategyFactoryLoader.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.executiongraph.failover.flip1; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.restartstrategy.RestartStrategies.FailureRateRestartStrategyConfiguration; +import org.apache.flink.api.common.restartstrategy.RestartStrategies.FallbackRestartStrategyConfiguration; +import org.apache.flink.api.common.restartstrategy.RestartStrategies.FixedDelayRestartStrategyConfiguration; +import org.apache.flink.api.common.restartstrategy.RestartStrategies.NoRestartStrategyConfiguration; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestartStrategyOptions; +import org.apache.flink.runtime.executiongraph.restart.NoOrFixedIfCheckpointingEnabledRestartStrategyFactory; +import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; + +import java.util.Optional; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A utility class to load {@link RestartBackoffTimeStrategy.Factory} from the configuration. + * It respects the configs for the legacy {@link RestartStrategy}. + */ +public final class RestartBackoffTimeStrategyFactoryLoader { + + private RestartBackoffTimeStrategyFactoryLoader() { + } + + /** + * Creates {@link RestartBackoffTimeStrategy.Factory} from the given configuration. + * + *

The strategy factory is decided in order as follows: + *

    + *
  1. Strategy set within job graph, i.e. {@link RestartStrategies.RestartStrategyConfiguration}, + * unless the config is {@link RestartStrategies.FallbackRestartStrategyConfiguration}.
  2. + *
  3. Strategy set in the cluster(server-side) config (flink-conf.yaml), + * unless the strategy is not specified
  4. + *
  5. {@link FixedDelayRestartBackoffTimeStrategy.FixedDelayRestartBackoffTimeStrategyFactory} if + * checkpointing is enabled. Otherwise {@link NoRestartBackoffTimeStrategy.NoRestartBackoffTimeStrategyFactory}
  6. + *
+ * + * @param jobRestartStrategyConfiguration restart configuration given within the job graph + * @param clusterConfiguration cluster(server-side) configuration + * @param isCheckpointingEnabled if checkpointing is enabled for the job + * @return new version restart strategy factory + */ + public static RestartBackoffTimeStrategy.Factory createRestartStrategyFactory( + final RestartStrategies.RestartStrategyConfiguration jobRestartStrategyConfiguration, + final Configuration clusterConfiguration, + final boolean isCheckpointingEnabled) { + + checkNotNull(jobRestartStrategyConfiguration); + checkNotNull(clusterConfiguration); + + return getJobRestartStrategyFactory(jobRestartStrategyConfiguration) + .orElse(getClusterRestartStrategyFactory(clusterConfiguration) + .orElse(getDefaultRestartStrategyFactory(isCheckpointingEnabled))); + } + + private static Optional getJobRestartStrategyFactory( + final RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration) { + + if (restartStrategyConfiguration instanceof NoRestartStrategyConfiguration) { + return Optional.of(NoRestartBackoffTimeStrategy.NoRestartBackoffTimeStrategyFactory.INSTANCE); + } else if (restartStrategyConfiguration instanceof FixedDelayRestartStrategyConfiguration) { + final FixedDelayRestartStrategyConfiguration fixedDelayConfig = + (FixedDelayRestartStrategyConfiguration) restartStrategyConfiguration; + + return Optional.of(new FixedDelayRestartBackoffTimeStrategy.FixedDelayRestartBackoffTimeStrategyFactory( + fixedDelayConfig.getRestartAttempts(), + fixedDelayConfig.getDelayBetweenAttemptsInterval().toMilliseconds())); + } else if (restartStrategyConfiguration instanceof FailureRateRestartStrategyConfiguration) { + final FailureRateRestartStrategyConfiguration failureRateConfig = + (FailureRateRestartStrategyConfiguration) restartStrategyConfiguration; + + return Optional.of(new FailureRateRestartBackoffTimeStrategy.FailureRateRestartBackoffTimeStrategyFactory( + failureRateConfig.getMaxFailureRate(), + failureRateConfig.getFailureInterval().toMilliseconds(), + failureRateConfig.getDelayBetweenAttemptsInterval().toMilliseconds())); + } else if (restartStrategyConfiguration instanceof FallbackRestartStrategyConfiguration) { + return Optional.empty(); + } else { + throw new IllegalArgumentException("Unknown restart strategy configuration " + + restartStrategyConfiguration + "."); + } + } + + private static Optional getClusterRestartStrategyFactory( + final Configuration clusterConfiguration) { + + final String restartStrategyName = clusterConfiguration.getString(RestartStrategyOptions.RESTART_STRATEGY); + if (restartStrategyName == null) { + return Optional.empty(); + } + + switch (restartStrategyName.toLowerCase()) { + case "none": + case "off": + case "disable": + return Optional.of(NoRestartBackoffTimeStrategy.NoRestartBackoffTimeStrategyFactory.INSTANCE); + case "fixeddelay": + case "fixed-delay": + return Optional.of(FixedDelayRestartBackoffTimeStrategy.createFactory(clusterConfiguration)); + case "failurerate": + case "failure-rate": + return Optional.of(FailureRateRestartBackoffTimeStrategy.createFactory(clusterConfiguration)); + default: + throw new IllegalArgumentException("Unknown restart strategy " + restartStrategyName + "."); + } + } + + private static RestartBackoffTimeStrategy.Factory getDefaultRestartStrategyFactory( + final boolean isCheckpointingEnabled) { + + if (isCheckpointingEnabled) { + // fixed delay restart strategy with default params + return new FixedDelayRestartBackoffTimeStrategy.FixedDelayRestartBackoffTimeStrategyFactory( + NoOrFixedIfCheckpointingEnabledRestartStrategyFactory.DEFAULT_RESTART_ATTEMPTS, + NoOrFixedIfCheckpointingEnabledRestartStrategyFactory.DEFAULT_RESTART_DELAY); + } else { + return NoRestartBackoffTimeStrategy.NoRestartBackoffTimeStrategyFactory.INSTANCE; + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/NoOrFixedIfCheckpointingEnabledRestartStrategyFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/NoOrFixedIfCheckpointingEnabledRestartStrategyFactory.java index dbdf63aee4..3e40acd6f7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/NoOrFixedIfCheckpointingEnabledRestartStrategyFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/NoOrFixedIfCheckpointingEnabledRestartStrategyFactory.java @@ -27,7 +27,9 @@ import java.time.Duration; public class NoOrFixedIfCheckpointingEnabledRestartStrategyFactory extends RestartStrategyFactory { private static final long serialVersionUID = -1809462525812787862L; - private static final long DEFAULT_RESTART_DELAY = Duration.ofSeconds(1L).toMillis(); + public static final int DEFAULT_RESTART_ATTEMPTS = Integer.MAX_VALUE; + + public static final long DEFAULT_RESTART_DELAY = Duration.ofSeconds(1L).toMillis(); @Override public RestartStrategy createRestartStrategy() { @@ -36,7 +38,7 @@ public class NoOrFixedIfCheckpointingEnabledRestartStrategyFactory extends Resta RestartStrategy createRestartStrategy(boolean isCheckpointingEnabled) { if (isCheckpointingEnabled) { - return new FixedDelayRestartStrategy(Integer.MAX_VALUE, DEFAULT_RESTART_DELAY); + return new FixedDelayRestartStrategy(DEFAULT_RESTART_ATTEMPTS, DEFAULT_RESTART_DELAY); } else { return new NoRestartStrategy(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartBackoffTimeStrategyFactoryLoaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartBackoffTimeStrategyFactoryLoaderTest.java new file mode 100644 index 0000000000..3114e76ba2 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartBackoffTimeStrategyFactoryLoaderTest.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.executiongraph.failover.flip1; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestartStrategyOptions; +import org.apache.flink.runtime.executiongraph.restart.NoOrFixedIfCheckpointingEnabledRestartStrategyFactory; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import static junit.framework.TestCase.assertEquals; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsInstanceOf.instanceOf; + +/** + * Unit tests for {@link RestartBackoffTimeStrategyFactoryLoader}. + */ +public class RestartBackoffTimeStrategyFactoryLoaderTest extends TestLogger { + + private static final RestartStrategies.RestartStrategyConfiguration DEFAULT_JOB_LEVEL_RESTART_CONFIGURATION = + RestartStrategies.fallBackRestart(); + + @Test + public void testNoRestartStrategySpecifiedInJobConfig() { + final Configuration conf = new Configuration(); + conf.setString(RestartStrategyOptions.RESTART_STRATEGY, "failure-rate"); + + final RestartBackoffTimeStrategy.Factory factory = + RestartBackoffTimeStrategyFactoryLoader.createRestartStrategyFactory( + RestartStrategies.noRestart(), + conf, + false); + + assertEquals(NoRestartBackoffTimeStrategy.NoRestartBackoffTimeStrategyFactory.INSTANCE, factory); + } + + @Test + public void testFixedDelayRestartStrategySpecifiedInJobConfig() { + final Configuration conf = new Configuration(); + conf.setString(RestartStrategyOptions.RESTART_STRATEGY, "failure-rate"); + + final RestartBackoffTimeStrategy.Factory factory = + RestartBackoffTimeStrategyFactoryLoader.createRestartStrategyFactory( + RestartStrategies.fixedDelayRestart(1, Time.milliseconds(1000)), + conf, + false); + + assertThat( + factory, + instanceOf(FixedDelayRestartBackoffTimeStrategy.FixedDelayRestartBackoffTimeStrategyFactory.class)); + } + + @Test + public void testFailureRateRestartStrategySpecifiedInJobConfig() { + final Configuration conf = new Configuration(); + conf.setString(RestartStrategyOptions.RESTART_STRATEGY, "fixed-delay"); + + final RestartBackoffTimeStrategy.Factory factory = + RestartBackoffTimeStrategyFactoryLoader.createRestartStrategyFactory( + RestartStrategies.failureRateRestart(1, Time.milliseconds(1000), Time.milliseconds(1000)), + conf, + false); + + assertThat( + factory, + instanceOf(FailureRateRestartBackoffTimeStrategy.FailureRateRestartBackoffTimeStrategyFactory.class)); + } + + @Test + public void testNoRestartStrategySpecifiedInClusterConfig() { + final Configuration conf = new Configuration(); + conf.setString(RestartStrategyOptions.RESTART_STRATEGY, "none"); + + final RestartBackoffTimeStrategy.Factory factory = + RestartBackoffTimeStrategyFactoryLoader.createRestartStrategyFactory( + DEFAULT_JOB_LEVEL_RESTART_CONFIGURATION, + conf, + false); + + assertEquals(NoRestartBackoffTimeStrategy.NoRestartBackoffTimeStrategyFactory.INSTANCE, factory); + } + + @Test + public void testFixedDelayStrategySpecifiedInClusterConfig() { + final Configuration conf = new Configuration(); + conf.setString(RestartStrategyOptions.RESTART_STRATEGY, "fixed-delay"); + + final RestartBackoffTimeStrategy.Factory factory = + RestartBackoffTimeStrategyFactoryLoader.createRestartStrategyFactory( + DEFAULT_JOB_LEVEL_RESTART_CONFIGURATION, + conf, + false); + + assertThat( + factory, + instanceOf(FixedDelayRestartBackoffTimeStrategy.FixedDelayRestartBackoffTimeStrategyFactory.class)); + } + + @Test + public void testFailureRateStrategySpecifiedInClusterConfig() { + final Configuration conf = new Configuration(); + conf.setString(RestartStrategyOptions.RESTART_STRATEGY, "failure-rate"); + + final RestartBackoffTimeStrategy.Factory factory = + RestartBackoffTimeStrategyFactoryLoader.createRestartStrategyFactory( + DEFAULT_JOB_LEVEL_RESTART_CONFIGURATION, + conf, + false); + + assertThat( + factory, + instanceOf(FailureRateRestartBackoffTimeStrategy.FailureRateRestartBackoffTimeStrategyFactory.class)); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidStrategySpecifiedInClusterConfig() { + final Configuration conf = new Configuration(); + conf.setString(RestartStrategyOptions.RESTART_STRATEGY, "invalid-strategy"); + + RestartBackoffTimeStrategyFactoryLoader.createRestartStrategyFactory( + DEFAULT_JOB_LEVEL_RESTART_CONFIGURATION, + conf, + false); + } + + @Test + public void testNoStrategySpecifiedWhenCheckpointingEnabled() { + final RestartBackoffTimeStrategy.Factory factory = + RestartBackoffTimeStrategyFactoryLoader.createRestartStrategyFactory( + DEFAULT_JOB_LEVEL_RESTART_CONFIGURATION, + new Configuration(), + true); + + RestartBackoffTimeStrategy strategy = factory.create(); + assertThat( + strategy, + instanceOf(FixedDelayRestartBackoffTimeStrategy.class)); + + FixedDelayRestartBackoffTimeStrategy fixedDelayStrategy = (FixedDelayRestartBackoffTimeStrategy) strategy; + assertEquals( + NoOrFixedIfCheckpointingEnabledRestartStrategyFactory.DEFAULT_RESTART_DELAY, + fixedDelayStrategy.getBackoffTime()); + assertEquals( + NoOrFixedIfCheckpointingEnabledRestartStrategyFactory.DEFAULT_RESTART_ATTEMPTS, + fixedDelayStrategy.getMaxNumberRestartAttempts()); + } + + @Test + public void testNoStrategySpecifiedWhenCheckpointingDisabled() { + final RestartBackoffTimeStrategy.Factory factory = + RestartBackoffTimeStrategyFactoryLoader.createRestartStrategyFactory( + DEFAULT_JOB_LEVEL_RESTART_CONFIGURATION, + new Configuration(), + false); + + assertThat( + factory, + instanceOf(NoRestartBackoffTimeStrategy.NoRestartBackoffTimeStrategyFactory.class)); + } +} -- Gitee From 63cdae9a8fbf139fce959e0d57029c39bd5b7333 Mon Sep 17 00:00:00 2001 From: "kevin.cyj" Date: Thu, 19 Sep 2019 20:48:31 +0800 Subject: [PATCH 043/268] [FLINK-14139][rest] Fix potential memory leak problem of rest server. This closes #9750. --- .../flink/runtime/rest/FileUploadHandler.java | 11 ++++++ .../runtime/rest/FileUploadHandlerTest.java | 34 +++++++++++++++++++ 2 files changed, 45 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java index 3cd973216e..6f60830b94 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FileUploadHandler.java @@ -38,6 +38,7 @@ import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseSt import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.LastHttpContent; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.Attribute; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.DefaultHttpDataFactory; +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.DiskAttribute; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.DiskFileUpload; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.HttpDataFactory; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.multipart.HttpPostRequestDecoder; @@ -84,7 +85,17 @@ public class FileUploadHandler extends SimpleChannelInboundHandler { public FileUploadHandler(final Path uploadDir) { super(true); + + // the clean up of temp files when jvm exits is handled by org.apache.flink.util.ShutdownHookUtil; thus, + // it's no need to register those files (post chunks and upload file chunks) to java.io.DeleteOnExitHook + // which may lead to memory leak. + DiskAttribute.deleteOnExitTemporaryFile = false; + DiskFileUpload.deleteOnExitTemporaryFile = false; + DiskFileUpload.baseDirectory = uploadDir.normalize().toAbsolutePath().toString(); + // share the same directory with file upload for post chunks storage. + DiskAttribute.baseDirectory = DiskFileUpload.baseDirectory; + this.uploadDir = requireNonNull(uploadDir); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java index 771fd8a837..80fa4b957d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/FileUploadHandlerTest.java @@ -38,8 +38,12 @@ import org.junit.Test; import java.io.File; import java.io.IOException; import java.io.StringWriter; +import java.lang.reflect.Field; +import java.util.LinkedHashSet; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * Tests for the {@link FileUploadHandler}. Ensures that multipart http messages containing files and/or json are properly @@ -136,6 +140,8 @@ public class FileUploadHandlerTest extends TestLogger { try (Response response = client.newCall(fileRequest).execute()) { assertEquals(fileHandler.getMessageHeaders().getResponseStatusCode().code(), response.code()); } + + verifyNoFileIsRegisteredToDeleteOnExitHook(); } @Test @@ -162,6 +168,8 @@ public class FileUploadHandlerTest extends TestLogger { assertEquals(mixedHandler.getMessageHeaders().getResponseStatusCode().code(), response.code()); assertEquals(json, mixedHandler.lastReceivedRequest); } + + verifyNoFileIsRegisteredToDeleteOnExitHook(); } @Test @@ -188,6 +196,8 @@ public class FileUploadHandlerTest extends TestLogger { // FileUploads are outright forbidden assertEquals(HttpResponseStatus.BAD_REQUEST.code(), response.code()); } + + verifyNoFileIsRegisteredToDeleteOnExitHook(); } @Test @@ -212,6 +222,8 @@ public class FileUploadHandlerTest extends TestLogger { // JSON payload did not match expected format assertEquals(HttpResponseStatus.BAD_REQUEST.code(), response.code()); } + + verifyNoFileIsRegisteredToDeleteOnExitHook(); } @Test @@ -223,6 +235,8 @@ public class FileUploadHandlerTest extends TestLogger { assertEquals(HttpResponseStatus.BAD_REQUEST.code(), response.code()); } MULTIPART_UPLOAD_RESOURCE.assertUploadDirectoryIsEmpty(); + + verifyNoFileIsRegisteredToDeleteOnExitHook(); } /** @@ -238,5 +252,25 @@ public class FileUploadHandlerTest extends TestLogger { assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR.code(), response.code()); } MULTIPART_UPLOAD_RESOURCE.assertUploadDirectoryIsEmpty(); + + verifyNoFileIsRegisteredToDeleteOnExitHook(); + } + + /** + * DiskAttribute and DiskFileUpload class of netty store post chunks and file chunks as temp files on local disk. + * By default, netty will register these temp files to java.io.DeleteOnExitHook which may lead to memory leak. + * {@link FileUploadHandler} disables the shutdown hook registration so no file should be registered. Note that + * clean up of temp files is handed over to {@link org.apache.flink.runtime.entrypoint.ClusterEntrypoint}. + */ + private void verifyNoFileIsRegisteredToDeleteOnExitHook() { + try { + Class clazz = Class.forName("java.io.DeleteOnExitHook"); + Field field = clazz.getDeclaredField("files"); + field.setAccessible(true); + LinkedHashSet files = (LinkedHashSet) field.get(null); + assertTrue(files.isEmpty()); + } catch (ClassNotFoundException | IllegalAccessException | NoSuchFieldException e) { + fail("This should never happen."); + } } } -- Gitee From 0c11dff4d73b6da64daf109af426e6d23dfc8ea6 Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Tue, 24 Sep 2019 17:05:22 +0800 Subject: [PATCH 044/268] [FLINK-14183] Remove scala duration usages from FutureUtils and other related classes --- .../LeaderRetrievalHandlerTest.java | 8 ++--- .../runtime/webmonitor/WebFrontendITCase.java | 29 ++++++++++--------- .../webmonitor/testutils/HttpTestClient.java | 13 ++++----- .../flink/runtime/concurrent/FutureUtils.java | 22 ++++---------- .../runtime/util/LeaderRetrievalUtils.java | 27 ++++------------- .../runtime/rpc/akka/AkkaRpcActorTest.java | 4 +-- 6 files changed, 37 insertions(+), 66 deletions(-) diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/LeaderRetrievalHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/LeaderRetrievalHandlerTest.java index d4ea516f4e..a99daf7730 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/LeaderRetrievalHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/LeaderRetrievalHandlerTest.java @@ -84,18 +84,18 @@ public class LeaderRetrievalHandlerTest extends TestLogger { try (HttpTestClient httpClient = new HttpTestClient("localhost", bootstrap.getServerPort())) { // 1. no leader gateway available --> Service unavailable - httpClient.sendGetRequest(restPath, FutureUtils.toFiniteDuration(timeout)); + httpClient.sendGetRequest(restPath, FutureUtils.toDuration(timeout)); - HttpTestClient.SimpleHttpResponse response = httpClient.getNextResponse(FutureUtils.toFiniteDuration(timeout)); + HttpTestClient.SimpleHttpResponse response = httpClient.getNextResponse(FutureUtils.toDuration(timeout)); Assert.assertEquals(HttpResponseStatus.SERVICE_UNAVAILABLE, response.getStatus()); // 2. with leader gatewayFuture.complete(gateway); - httpClient.sendGetRequest(restPath, FutureUtils.toFiniteDuration(timeout)); + httpClient.sendGetRequest(restPath, FutureUtils.toDuration(timeout)); - response = httpClient.getNextResponse(FutureUtils.toFiniteDuration(timeout)); + response = httpClient.getNextResponse(FutureUtils.toDuration(timeout)); Assert.assertEquals(HttpResponseStatus.OK, response.getStatus()); Assert.assertEquals(RESPONSE_MESSAGE, response.getContent()); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java index abeb9ea31a..ffee6f1ddd 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java @@ -52,16 +52,14 @@ import java.io.InputStream; import java.net.HttpURLConnection; import java.net.URL; import java.nio.file.Files; +import java.time.Duration; +import java.time.LocalTime; import java.util.Collection; import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import scala.concurrent.duration.Deadline; -import scala.concurrent.duration.FiniteDuration; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -257,13 +255,13 @@ public class WebFrontendITCase extends TestLogger { // wait for tasks to be properly running BlockingInvokable.latch.await(); - final FiniteDuration testTimeout = new FiniteDuration(2, TimeUnit.MINUTES); - final Deadline deadline = testTimeout.fromNow(); + final Duration testTimeout = Duration.ofMinutes(2); + final LocalTime deadline = LocalTime.now().plus(testTimeout); try (HttpTestClient client = new HttpTestClient("localhost", getRestPort())) { // cancel the job - client.sendPatchRequest("/jobs/" + jid + "/", deadline.timeLeft()); - HttpTestClient.SimpleHttpResponse response = client.getNextResponse(deadline.timeLeft()); + client.sendPatchRequest("/jobs/" + jid + "/", getTimeLeft(deadline)); + HttpTestClient.SimpleHttpResponse response = client.getNextResponse(getTimeLeft(deadline)); assertEquals(HttpResponseStatus.ACCEPTED, response.getStatus()); assertEquals("application/json; charset=UTF-8", response.getType()); @@ -277,7 +275,7 @@ public class WebFrontendITCase extends TestLogger { // ensure we can access job details when its finished (FLINK-4011) try (HttpTestClient client = new HttpTestClient("localhost", getRestPort())) { - FiniteDuration timeout = new FiniteDuration(30, TimeUnit.SECONDS); + Duration timeout = Duration.ofSeconds(30); client.sendGetRequest("/jobs/" + jid + "/config", timeout); HttpTestClient.SimpleHttpResponse response = client.getNextResponse(timeout); @@ -316,15 +314,14 @@ public class WebFrontendITCase extends TestLogger { // wait for tasks to be properly running BlockingInvokable.latch.await(); - final FiniteDuration testTimeout = new FiniteDuration(2, TimeUnit.MINUTES); - final Deadline deadline = testTimeout.fromNow(); + final Duration testTimeout = Duration.ofMinutes(2); + final LocalTime deadline = LocalTime.now().plus(testTimeout); try (HttpTestClient client = new HttpTestClient("localhost", getRestPort())) { // Request the file from the web server - client.sendGetRequest("/jobs/" + jid + "/yarn-cancel", deadline.timeLeft()); + client.sendGetRequest("/jobs/" + jid + "/yarn-cancel", getTimeLeft(deadline)); - HttpTestClient.SimpleHttpResponse response = client - .getNextResponse(deadline.timeLeft()); + HttpTestClient.SimpleHttpResponse response = client.getNextResponse(getTimeLeft(deadline)); assertEquals(HttpResponseStatus.ACCEPTED, response.getStatus()); assertEquals("application/json; charset=UTF-8", response.getType()); @@ -347,6 +344,10 @@ public class WebFrontendITCase extends TestLogger { .collect(Collectors.toList()); } + private static Duration getTimeLeft(LocalTime deadline) { + return Duration.between(LocalTime.now(), deadline); + } + /** * Test invokable that allows waiting for all subtasks to be running. */ diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java index d94f7a265e..0d51cc4b08 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java @@ -47,13 +47,12 @@ import org.apache.flink.shaded.netty4.io.netty.util.CharsetUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Duration; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import scala.concurrent.duration.FiniteDuration; - /** * A simple HTTP client. * @@ -130,7 +129,7 @@ public class HttpTestClient implements AutoCloseable { * * @param request The {@link HttpRequest} to send to the server */ - public void sendRequest(HttpRequest request, FiniteDuration timeout) throws InterruptedException, TimeoutException { + public void sendRequest(HttpRequest request, Duration timeout) throws InterruptedException, TimeoutException { LOG.debug("Writing {}.", request); // Make the connection attempt. @@ -153,7 +152,7 @@ public class HttpTestClient implements AutoCloseable { * * @param path The $path to GET (http://$host:$host/$path) */ - public void sendGetRequest(String path, FiniteDuration timeout) throws TimeoutException, InterruptedException { + public void sendGetRequest(String path, Duration timeout) throws TimeoutException, InterruptedException { if (!path.startsWith("/")) { path = "/" + path; } @@ -172,7 +171,7 @@ public class HttpTestClient implements AutoCloseable { * * @param path The $path to DELETE (http://$host:$host/$path) */ - public void sendDeleteRequest(String path, FiniteDuration timeout) throws TimeoutException, InterruptedException { + public void sendDeleteRequest(String path, Duration timeout) throws TimeoutException, InterruptedException { if (!path.startsWith("/")) { path = "/" + path; } @@ -191,7 +190,7 @@ public class HttpTestClient implements AutoCloseable { * * @param path The $path to PATCH (http://$host:$host/$path) */ - public void sendPatchRequest(String path, FiniteDuration timeout) throws TimeoutException, InterruptedException { + public void sendPatchRequest(String path, Duration timeout) throws TimeoutException, InterruptedException { if (!path.startsWith("/")) { path = "/" + path; } @@ -221,7 +220,7 @@ public class HttpTestClient implements AutoCloseable { * @param timeout Timeout in milliseconds for the next response to become available * @return The next available {@link SimpleHttpResponse} */ - public SimpleHttpResponse getNextResponse(FiniteDuration timeout) throws InterruptedException, + public SimpleHttpResponse getNextResponse(Duration timeout) throws InterruptedException, TimeoutException { SimpleHttpResponse response = responses.poll(timeout.toMillis(), TimeUnit.MILLISECONDS); 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 5b958b7450..b9e5ed463e 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 @@ -30,6 +30,7 @@ import akka.dispatch.OnComplete; import javax.annotation.Nonnull; +import java.time.Duration; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -55,7 +56,6 @@ import java.util.function.Predicate; import java.util.function.Supplier; import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -833,23 +833,13 @@ public class FutureUtils { } /** - * Converts Flink time into a {@link FiniteDuration}. + * Converts Flink time into a {@link Duration}. * - * @param time to convert into a FiniteDuration - * @return FiniteDuration with the length of the given time + * @param time to convert into a Duration + * @return Duration with the length of the given time */ - public static FiniteDuration toFiniteDuration(Time time) { - return new FiniteDuration(time.toMilliseconds(), TimeUnit.MILLISECONDS); - } - - /** - * Converts {@link FiniteDuration} into Flink time. - * - * @param finiteDuration to convert into Flink time - * @return Flink time with the length of the given finite duration - */ - public static Time toTime(FiniteDuration finiteDuration) { - return Time.milliseconds(finiteDuration.toMillis()); + public static Duration toDuration(Time time) { + return Duration.ofMillis(time.toMilliseconds()); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java index e0dca79ff1..2b71c70b0d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java @@ -31,8 +31,8 @@ import org.slf4j.LoggerFactory; import java.net.InetAddress; import java.util.UUID; +import java.util.concurrent.TimeUnit; -import scala.concurrent.Await; import scala.concurrent.Future; import scala.concurrent.Promise; import scala.concurrent.duration.FiniteDuration; @@ -58,24 +58,7 @@ public class LeaderRetrievalUtils { public static LeaderConnectionInfo retrieveLeaderConnectionInfo( LeaderRetrievalService leaderRetrievalService, Time timeout) throws LeaderRetrievalException { - return retrieveLeaderConnectionInfo(leaderRetrievalService, FutureUtils.toFiniteDuration(timeout)); - } - /** - * Retrieves the leader akka url and the current leader session ID. The values are stored in a - * {@link LeaderConnectionInfo} instance. - * - * @param leaderRetrievalService Leader retrieval service to retrieve the leader connection - * information - * @param timeout Timeout when to give up looking for the leader - * @return LeaderConnectionInfo containing the leader's akka URL and the current leader session - * ID - * @throws LeaderRetrievalException - */ - public static LeaderConnectionInfo retrieveLeaderConnectionInfo( - LeaderRetrievalService leaderRetrievalService, - FiniteDuration timeout - ) throws LeaderRetrievalException { LeaderConnectionInfoListener listener = new LeaderConnectionInfoListener(); try { @@ -83,10 +66,10 @@ public class LeaderRetrievalUtils { Future connectionInfoFuture = listener.getLeaderConnectionInfoFuture(); - return Await.result(connectionInfoFuture, timeout); + return FutureUtils.toJava(connectionInfoFuture).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } catch (Exception e) { throw new LeaderRetrievalException("Could not retrieve the leader address and leader " + - "session ID.", e); + "session ID.", e); } finally { try { leaderRetrievalService.stop(); @@ -160,9 +143,9 @@ public class LeaderRetrievalUtils { } } } - + // ------------------------------------------------------------------------ - + /** * Private constructor to prevent instantiation. */ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java index c010810ef5..0597b01f46 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -51,8 +51,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; -import scala.concurrent.Await; - import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -272,7 +270,7 @@ public class AkkaRpcActorTest extends TestLogger { terminationFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } finally { rpcActorSystem.terminate(); - Await.ready(rpcActorSystem.whenTerminated(), FutureUtils.toFiniteDuration(timeout)); + FutureUtils.toJava(rpcActorSystem.whenTerminated()).get(timeout.getSize(), timeout.getUnit()); } } -- Gitee From d528ea0b7d0b292dcfc8b07e723beeea06774f38 Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Tue, 24 Sep 2019 18:08:35 +0800 Subject: [PATCH 045/268] [FLINK-14183] Remove scala duration usages from LeaderRetrievalUtils, ConnectionUtils and other related classes --- .../flink/runtime/net/ConnectionUtils.java | 13 +++++------- .../taskexecutor/TaskManagerRunner.java | 3 ++- .../runtime/util/LeaderRetrievalUtils.java | 20 +++++++------------ .../dispatcher/ZooKeeperHADispatcherTest.java | 5 ++++- .../LeaderChangeClusterComponentsTest.java | 7 +++++-- .../ZooKeeperLeaderRetrievalTest.java | 12 +++++------ .../ProcessFailureCancelingITCase.java | 4 +++- 7 files changed, 31 insertions(+), 33 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java index 9ee5962047..cf1a815f2d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java @@ -32,14 +32,12 @@ import java.net.NetworkInterface; import java.net.Socket; import java.net.SocketAddress; import java.net.UnknownHostException; +import java.time.Duration; import java.util.Arrays; import java.util.Collections; import java.util.Enumeration; import java.util.List; import java.util.UUID; -import java.util.concurrent.TimeUnit; - -import scala.concurrent.duration.FiniteDuration; /** * Utilities to determine the network interface and address that should be used to bind the @@ -342,7 +340,7 @@ public class ConnectionUtils { */ public static class LeaderConnectingAddressListener implements LeaderRetrievalListener { - private static final FiniteDuration defaultLoggingDelay = new FiniteDuration(400, TimeUnit.MILLISECONDS); + private static final Duration defaultLoggingDelay = Duration.ofMillis(400); private enum LeaderRetrievalState { NOT_RETRIEVED, @@ -357,14 +355,13 @@ public class ConnectionUtils { private Exception exception; public InetAddress findConnectingAddress( - FiniteDuration timeout) throws LeaderRetrievalException { + Duration timeout) throws LeaderRetrievalException { return findConnectingAddress(timeout, defaultLoggingDelay); } public InetAddress findConnectingAddress( - FiniteDuration timeout, - FiniteDuration startLoggingAfter) - throws LeaderRetrievalException { + Duration timeout, + Duration startLoggingAfter) throws LeaderRetrievalException { final long startTimeNanos = System.nanoTime(); long currentSleepTime = MIN_SLEEP_TIME; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index 72cc2c9b21..6493decef9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -69,6 +69,7 @@ import org.slf4j.LoggerFactory; import java.lang.reflect.UndeclaredThrowableException; import java.net.InetAddress; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.concurrent.Callable; @@ -426,7 +427,7 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync final Configuration configuration, final HighAvailabilityServices haServices) throws LeaderRetrievalException { - final Time lookupTimeout = Time.milliseconds(AkkaUtils.getLookupTimeout(configuration).toMillis()); + final Duration lookupTimeout = AkkaUtils.getLookupTimeout(configuration); final InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress( haServices.getResourceManagerLeaderRetriever(), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java index 2b71c70b0d..e3b2e27228 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.util; -import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; @@ -30,12 +29,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.net.InetAddress; +import java.time.Duration; import java.util.UUID; import java.util.concurrent.TimeUnit; import scala.concurrent.Future; import scala.concurrent.Promise; -import scala.concurrent.duration.FiniteDuration; /** * Utility class to work with {@link LeaderRetrievalService} class. @@ -57,7 +56,7 @@ public class LeaderRetrievalUtils { */ public static LeaderConnectionInfo retrieveLeaderConnectionInfo( LeaderRetrievalService leaderRetrievalService, - Time timeout) throws LeaderRetrievalException { + Duration timeout) throws LeaderRetrievalException { LeaderConnectionInfoListener listener = new LeaderConnectionInfoListener(); @@ -66,7 +65,7 @@ public class LeaderRetrievalUtils { Future connectionInfoFuture = listener.getLeaderConnectionInfoFuture(); - return FutureUtils.toJava(connectionInfoFuture).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + return FutureUtils.toJava(connectionInfoFuture).get(timeout.toMillis(), TimeUnit.MILLISECONDS); } catch (Exception e) { throw new LeaderRetrievalException("Could not retrieve the leader address and leader " + "session ID.", e); @@ -81,28 +80,23 @@ public class LeaderRetrievalUtils { public static InetAddress findConnectingAddress( LeaderRetrievalService leaderRetrievalService, - Time timeout) throws LeaderRetrievalException { - return findConnectingAddress(leaderRetrievalService, new FiniteDuration(timeout.getSize(), timeout.getUnit())); - } + Duration timeout) throws LeaderRetrievalException { - public static InetAddress findConnectingAddress( - LeaderRetrievalService leaderRetrievalService, - FiniteDuration timeout) throws LeaderRetrievalException { ConnectionUtils.LeaderConnectingAddressListener listener = new ConnectionUtils.LeaderConnectingAddressListener(); try { leaderRetrievalService.start(listener); LOG.info("Trying to select the network interface and address to use " + - "by connecting to the leading JobManager."); + "by connecting to the leading JobManager."); LOG.info("TaskManager will try to connect for " + timeout + - " before falling back to heuristics"); + " before falling back to heuristics"); return listener.findConnectingAddress(timeout); } catch (Exception e) { throw new LeaderRetrievalException("Could not find the connecting address by " + - "connecting to the current leader.", e); + "connecting to the current leader.", e); } finally { try { leaderRetrievalService.stop(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java index dd535160b7..0343caadf0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java @@ -62,6 +62,7 @@ import org.junit.rules.TestName; import javax.annotation.Nonnull; import java.io.IOException; +import java.time.Duration; import java.util.Collection; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -293,7 +294,9 @@ public class ZooKeeperHADispatcherTest extends TestLogger { dispatcher1.start(); dispatcher2.start(); - final LeaderConnectionInfo leaderConnectionInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo(haServices.getDispatcherLeaderRetriever(), TIMEOUT); + final LeaderConnectionInfo leaderConnectionInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo( + haServices.getDispatcherLeaderRetriever(), + Duration.ofMillis(TIMEOUT.toMilliseconds())); final DispatcherGateway dispatcherGateway = rpcService.connect(leaderConnectionInfo.getAddress(), DispatcherId.fromUuid(leaderConnectionInfo.getLeaderSessionID()), DispatcherGateway.class).get(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java index 1d62b19f5f..cd9b047705 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java @@ -21,7 +21,6 @@ package org.apache.flink.runtime.leaderelection; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.api.common.time.Deadline; -import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.highavailability.nonha.embedded.TestingEmbeddedHaServices; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -157,7 +156,11 @@ public class LeaderChangeClusterComponentsTest extends TestLogger { highAvailabilityServices.grantResourceManagerLeadership(); // wait for the ResourceManager to confirm the leadership - assertThat(LeaderRetrievalUtils.retrieveLeaderConnectionInfo(highAvailabilityServices.getResourceManagerLeaderRetriever(), Time.minutes(TESTING_TIMEOUT.toMinutes())).getLeaderSessionID(), is(notNullValue())); + assertThat( + LeaderRetrievalUtils.retrieveLeaderConnectionInfo( + highAvailabilityServices.getResourceManagerLeaderRetriever(), + TESTING_TIMEOUT).getLeaderSessionID(), + is(notNullValue())); waitUntilTaskExecutorsHaveConnected(NUM_TMS, deadline); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java index d4e5dd98e9..d98292b90b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java @@ -45,9 +45,7 @@ import java.net.ServerSocket; import java.net.Socket; import java.net.SocketAddress; import java.net.UnknownHostException; -import java.util.concurrent.TimeUnit; - -import scala.concurrent.duration.FiniteDuration; +import java.time.Duration; import static org.junit.Assert.assertEquals; @@ -102,7 +100,7 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{ */ @Test public void testConnectingAddressRetrievalWithDelayedLeaderElection() throws Exception { - FiniteDuration timeout = new FiniteDuration(1, TimeUnit.MINUTES); + Duration timeout = Duration.ofMinutes(1L); long sleepingTime = 1000; @@ -197,7 +195,7 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{ */ @Test public void testTimeoutOfFindConnectingAddress() throws Exception { - FiniteDuration timeout = new FiniteDuration(1L, TimeUnit.SECONDS); + Duration timeout = Duration.ofSeconds(1L); LeaderRetrievalService leaderRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID); InetAddress result = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, timeout); @@ -207,14 +205,14 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{ static class FindConnectingAddress implements Runnable { - private final FiniteDuration timeout; + private final Duration timeout; private final LeaderRetrievalService leaderRetrievalService; private InetAddress result; private Exception exception; public FindConnectingAddress( - FiniteDuration timeout, + Duration timeout, LeaderRetrievalService leaderRetrievalService) { this.timeout = timeout; this.leaderRetrievalService = leaderRetrievalService; 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 0a1712ecf9..46f9a09e14 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 @@ -265,7 +265,9 @@ public class ProcessFailureCancelingITCase extends TestLogger { * @throws Exception if something goes wrong */ static DispatcherGateway retrieveDispatcherGateway(RpcService rpcService, HighAvailabilityServices haServices) throws Exception { - final LeaderConnectionInfo leaderConnectionInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo(haServices.getDispatcherLeaderRetriever(), Time.seconds(10L)); + final LeaderConnectionInfo leaderConnectionInfo = LeaderRetrievalUtils.retrieveLeaderConnectionInfo( + haServices.getDispatcherLeaderRetriever(), + Duration.ofSeconds(10L)); return rpcService.connect( leaderConnectionInfo.getAddress(), -- Gitee From e8ced9a39b5786fdc020c2134c83bf089f106270 Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Tue, 24 Sep 2019 17:06:54 +0800 Subject: [PATCH 046/268] [FLINK-14183] Remove scala duration usages from TestingUtils --- .../apache/flink/runtime/testingUtils/TestingUtils.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala index fdc2c07ba7..93aebcfea6 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala @@ -18,6 +18,7 @@ package org.apache.flink.runtime.testingUtils +import java.time.Duration import java.util import java.util.Collections import java.util.concurrent._ @@ -28,7 +29,6 @@ import org.apache.flink.api.common.time.Time import org.apache.flink.runtime.akka.AkkaUtils import org.apache.flink.runtime.concurrent.{ScheduledExecutor, ScheduledExecutorServiceAdapter} -import scala.concurrent.duration.{TimeUnit, _} import scala.concurrent.{ExecutionContext, ExecutionContextExecutor} import scala.language.postfixOps @@ -41,9 +41,9 @@ object TestingUtils { val testConfig = ConfigFactory.parseString(getDefaultTestingActorSystemConfigString) - val TESTING_DURATION = 2 minute + val TESTING_DURATION = Duration.ofMinutes(2L); - val TESTING_TIMEOUT = 1 minute + val TESTING_TIMEOUT = Duration.ofMinutes(1L); val TIMEOUT = Time.minutes(1L) -- Gitee From 81820806c899ced8c17fef552f8a3a35d2b08616 Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Tue, 24 Sep 2019 17:07:33 +0800 Subject: [PATCH 047/268] [FLINK-14183] Remove scala duration usages from MetricRegistryImplTest This closes #9757. --- .../apache/flink/runtime/metrics/MetricRegistryImplTest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java index 6089c936bd..0f69aca2d2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java @@ -45,13 +45,12 @@ import org.apache.flink.util.TestLogger; import org.junit.Assert; import org.junit.Test; +import java.time.Duration; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; -import scala.concurrent.duration.FiniteDuration; - import static org.apache.flink.util.Preconditions.checkNotNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -366,7 +365,7 @@ public class MetricRegistryImplTest extends TestLogger { */ @Test public void testQueryActorShutdown() throws Exception { - final FiniteDuration timeout = new FiniteDuration(10L, TimeUnit.SECONDS); + final Duration timeout = Duration.ofSeconds(10L); MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); -- Gitee From ca56b63c37176dabf163b745d6ee015012531243 Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Wed, 25 Sep 2019 22:41:58 +0800 Subject: [PATCH 048/268] [FLINK-14114][client] Shift down ClusterClient#timeout to RestClusterClient This closes #9723. --- .../org/apache/flink/client/program/ClusterClient.java | 7 +------ .../flink/client/program/rest/RestClusterClient.java | 8 ++++++++ 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index a61bae220a..8179403212 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -33,7 +33,6 @@ import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.StreamingPlan; import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; @@ -52,7 +51,6 @@ import javax.annotation.Nullable; import java.net.URISyntaxException; import java.net.URL; -import java.time.Duration; import java.util.Collection; import java.util.List; import java.util.Map; @@ -71,10 +69,8 @@ public abstract class ClusterClient implements AutoCloseable { final Optimizer compiler; /** Configuration of the client. */ - protected final Configuration flinkConfig; + private final Configuration flinkConfig; - /** Timeout for futures. */ - protected final Duration timeout; /** * For interactive invocations, the job results are only available after the ContextEnvironment has * been run inside the user JAR. We pass the Client to every instance of the ContextEnvironment @@ -99,7 +95,6 @@ public abstract class ClusterClient implements AutoCloseable { public ClusterClient(Configuration flinkConfig) { this.flinkConfig = Preconditions.checkNotNull(flinkConfig); this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), flinkConfig); - this.timeout = AkkaUtils.getClientTimeout(flinkConfig); } /** diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index 0303620e77..ecd403a07c 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -32,6 +32,7 @@ import org.apache.flink.client.program.rest.retry.ExponentialWaitStrategy; import org.apache.flink.client.program.rest.retry.WaitStrategy; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.client.JobSubmissionException; @@ -108,6 +109,7 @@ import java.net.MalformedURLException; import java.net.URL; import java.nio.file.Files; import java.nio.file.Paths; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -131,6 +133,9 @@ public class RestClusterClient extends ClusterClient { private final RestClusterClientConfiguration restClusterClientConfiguration; + /** Timeout for futures. */ + private final Duration timeout; + private final RestClient restClient; private final ExecutorService executorService = Executors.newFixedThreadPool(4, new ExecutorThreadFactory("Flink-RestClusterClient-IO")); @@ -163,6 +168,9 @@ public class RestClusterClient extends ClusterClient { T clusterId, WaitStrategy waitStrategy) throws Exception { super(configuration); + + this.timeout = AkkaUtils.getClientTimeout(configuration); + this.restClusterClientConfiguration = RestClusterClientConfiguration.fromConfiguration(configuration); if (restClient != null) { -- Gitee From 2b608e3169ee416c922a8337c2ba5fa873516fde Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Mon, 26 Aug 2019 15:24:30 +0800 Subject: [PATCH 049/268] [FLINK-14016][python][flink-table-planner] Introduce DataStreamPythonCalc for Python function execution * Introduces DataStreamPythonCalc for Python ScalarFunction execution * Introduces PythonScalarFunctionSplitRule which split multiple ScalarFunctions contained in the same node * Add a logical rewrite phase for Python related rules This closes #9748. --- .../table/functions/FunctionDefinition.java | 7 + .../table/functions/FunctionLanguage.java | 32 +++ .../flink/table/calcite/CalciteConfig.scala | 31 +++ .../apache/flink/table/plan/Optimizer.scala | 39 +++ .../flink/table/plan/StreamOptimizer.scala | 4 +- .../flink/table/plan/nodes/CommonCalc.scala | 1 - .../plan/nodes/dataset/DataSetCalc.scala | 1 - .../nodes/datastream/DataStreamCalc.scala | 43 +--- .../nodes/datastream/DataStreamCalcBase.scala | 68 +++++ .../datastream/DataStreamPythonCalc.scala | 68 +++++ .../table/plan/rules/FlinkRuleSets.scala | 10 +- .../rules/datastream/DataStreamCalcRule.scala | 15 +- .../datastream/DataStreamPythonCalcRule.scala | 64 +++++ .../PythonScalarFunctionSplitRule.scala | 211 ++++++++++++++++ .../flink/table/plan/util/PythonUtil.scala | 68 +++++ .../calcite/CalciteConfigBuilderTest.scala | 55 ++++ .../PythonScalarFunctionSplitRuleTest.scala | 238 ++++++++++++++++++ 17 files changed, 914 insertions(+), 41 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionLanguage.java create mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalcBase.scala create mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala create mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCalcRule.scala create mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonScalarFunctionSplitRule.scala create mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/PythonUtil.scala create mode 100644 flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonScalarFunctionSplitRuleTest.scala diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionDefinition.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionDefinition.java index 16379285f2..e43e84121c 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionDefinition.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionDefinition.java @@ -40,6 +40,13 @@ public interface FunctionDefinition { */ FunctionKind getKind(); + /** + * Returns the language of function this definition describes. + */ + default FunctionLanguage getLanguage() { + return FunctionLanguage.JVM; + } + /** * Returns the set of requirements this definition demands. */ diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionLanguage.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionLanguage.java new file mode 100644 index 0000000000..7b36b132a4 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionLanguage.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.table.functions; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * Categorizes the language of a {@link FunctionDefinition}. + */ +@PublicEvolving +public enum FunctionLanguage { + + JVM, + + PYTHON +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala index 8064fa9b9a..4ba17131dd 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala @@ -52,6 +52,12 @@ class CalciteConfigBuilder { private var replaceLogicalOptRules: Boolean = false private var logicalOptRuleSets: List[RuleSet] = Nil + /** + * Defines the logical rewrite rule set. + */ + private var replaceLogicalRewriteRules: Boolean = false + private var logicalRewriteRuleSets: List[RuleSet] = Nil + /** * Defines the physical optimization rule set. */ @@ -119,6 +125,25 @@ class CalciteConfigBuilder { this } + /** + * Replaces the built-in logical rewrite rule set with the given rule set. + */ + def replaceLogicalRewriteRuleSet(replaceRuleSet: RuleSet): CalciteConfigBuilder = { + Preconditions.checkNotNull(replaceRuleSet) + logicalRewriteRuleSets = List(replaceRuleSet) + replaceLogicalRewriteRules = true + this + } + + /** + * Appends the given logical rewrite rule set to the built-in rule set. + */ + def addLogicalRewriteRuleSet(addedRuleSet: RuleSet): CalciteConfigBuilder = { + Preconditions.checkNotNull(addedRuleSet) + logicalRewriteRuleSets = addedRuleSet :: logicalRewriteRuleSets + this + } + /** * Replaces the built-in optimization rule set with the given rule set. */ @@ -225,6 +250,8 @@ class CalciteConfigBuilder { replaceNormRules, getRuleSet(logicalOptRuleSets), replaceLogicalOptRules, + getRuleSet(logicalRewriteRuleSets), + replaceLogicalRewriteRules, getRuleSet(physicalOptRuleSets), replacePhysicalOptRules, getRuleSet(decoRuleSets), @@ -254,6 +281,10 @@ class CalciteConfig( val logicalOptRuleSet: Option[RuleSet], /** Whether this configuration replaces the built-in logical optimization rule set. */ val replacesLogicalOptRuleSet: Boolean, + /** A custom logical rewrite rule set. */ + val logicalRewriteRuleSet: Option[RuleSet], + /** Whether this configuration replaces the built-in logical rewrite rule set. */ + val replacesLogicalRewriteRuleSet: Boolean, /** A custom physical optimization rule set. */ val physicalOptRuleSet: Option[RuleSet], /** Whether this configuration replaces the built-in physical optimization rule set. */ diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/Optimizer.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/Optimizer.scala index 8506749dbe..4336c074c3 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/Optimizer.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/Optimizer.scala @@ -86,6 +86,25 @@ abstract class Optimizer( } } + /** + * Returns the logical rewrite rule set for this optimizer + * including a custom RuleSet configuration. + */ + protected def getLogicalRewriteRuleSet: RuleSet = { + materializedConfig.logicalRewriteRuleSet match { + + case None => + getBuiltInLogicalRewriteRuleSet + + case Some(ruleSet) => + if (materializedConfig.replacesLogicalRewriteRuleSet) { + ruleSet + } else { + RuleSets.ofList((getBuiltInLogicalRewriteRuleSet.asScala ++ ruleSet.asScala).asJava) + } + } + } + /** * Returns the physical optimization rule set for this optimizer * including a custom RuleSet configuration. @@ -117,6 +136,13 @@ abstract class Optimizer( FlinkRuleSets.LOGICAL_OPT_RULES } + /** + * Returns the built-in logical rewrite rules that are defined by the optimizer. + */ + protected def getBuiltInLogicalRewriteRuleSet: RuleSet = { + FlinkRuleSets.LOGICAL_REWRITE_RULES + } + /** * Returns the built-in physical optimization rules that are defined by the optimizer. */ @@ -153,6 +179,19 @@ abstract class Optimizer( } } + protected def optimizeLogicalRewritePlan(relNode: RelNode): RelNode = { + val logicalRewriteRuleSet = getLogicalRewriteRuleSet + if (logicalRewriteRuleSet.iterator().hasNext) { + runHepPlannerSimultaneously( + HepMatchOrder.TOP_DOWN, + logicalRewriteRuleSet, + relNode, + relNode.getTraitSet) + } else { + relNode + } + } + protected def optimizeLogicalPlan(relNode: RelNode): RelNode = { val logicalOptRuleSet = getLogicalOptRuleSet val logicalOutputProps = relNode.getTraitSet.replace(FlinkConventions.LOGICAL).simplify() diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/StreamOptimizer.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/StreamOptimizer.scala index 30ca4861ca..1ef6d7017d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/StreamOptimizer.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/StreamOptimizer.scala @@ -64,8 +64,8 @@ class StreamOptimizer( RelTimeIndicatorConverter.convert(decorPlan, relBuilder.getRexBuilder) val normalizedPlan = optimizeNormalizeLogicalPlan(planWithMaterializedTimeAttributes) val logicalPlan = optimizeLogicalPlan(normalizedPlan) - - val physicalPlan = optimizePhysicalPlan(logicalPlan, FlinkConventions.DATASTREAM) + val logicalRewritePlan = optimizeLogicalRewritePlan(logicalPlan) + val physicalPlan = optimizePhysicalPlan(logicalRewritePlan, FlinkConventions.DATASTREAM) optimizeDecoratePlan(physicalPlan, updatesAsRetraction) } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala index 36df67a072..34f4ba84bd 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala @@ -34,7 +34,6 @@ trait CommonCalc { private[flink] def generateFunction[T <: Function]( generator: FunctionCodeGenerator, ruleDescription: String, - inputSchema: RowSchema, returnSchema: RowSchema, calcProjection: Seq[RexNode], calcCondition: Option[RexNode], diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala index fd60bfe99c..7d2aa58a63 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala @@ -105,7 +105,6 @@ class DataSetCalc( val genFunction = generateFunction( generator, ruleDescription, - new RowSchema(getInput.getRowType), new RowSchema(getRowType), projection, condition, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala index 07b53eb9cb..f3e6afa7e4 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala @@ -18,18 +18,15 @@ package org.apache.flink.table.plan.nodes.datastream -import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} -import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.core.Calc -import org.apache.calcite.rel.metadata.RelMetadataQuery -import org.apache.calcite.rel.{RelNode, RelWriter} +import org.apache.calcite.rel.RelNode import org.apache.calcite.rex.RexProgram import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.calcite.RelTimeIndicatorConverter import org.apache.flink.table.codegen.FunctionCodeGenerator -import org.apache.flink.table.plan.nodes.CommonCalc import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.planner.StreamPlanner import org.apache.flink.table.runtime.CRowProcessRunner @@ -49,11 +46,14 @@ class DataStreamCalc( schema: RowSchema, calcProgram: RexProgram, ruleDescription: String) - extends Calc(cluster, traitSet, input, calcProgram) - with CommonCalc - with DataStreamRel { - - override def deriveRowType(): RelDataType = schema.relDataType + extends DataStreamCalcBase( + cluster, + traitSet, + input, + inputSchema, + schema, + calcProgram, + ruleDescription) { override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = { new DataStreamCalc( @@ -66,28 +66,6 @@ class DataStreamCalc( ruleDescription) } - override def toString: String = calcToString(calcProgram, getExpressionString) - - override def explainTerms(pw: RelWriter): RelWriter = { - pw.input("input", getInput) - .item("select", selectionToString(calcProgram, getExpressionString)) - .itemIf("where", - conditionToString(calcProgram, getExpressionString), - calcProgram.getCondition != null) - } - - override def computeSelfCost(planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = { - val child = this.getInput - val rowCnt = metadata.getRowCount(child) - computeSelfCost(calcProgram, planner, rowCnt) - } - - override def estimateRowCount(metadata: RelMetadataQuery): Double = { - val child = this.getInput - val rowCnt = metadata.getRowCount(child) - estimateRowCount(calcProgram, rowCnt) - } - override def translateToPlan( planner: StreamPlanner, queryConfig: StreamQueryConfig): DataStream[CRow] = { @@ -117,7 +95,6 @@ class DataStreamCalc( val genFunction = generateFunction( generator, ruleDescription, - inputSchema, schema, projection, condition, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalcBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalcBase.scala new file mode 100644 index 0000000000..5e62a9f2dd --- /dev/null +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalcBase.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.nodes.datastream + +import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rel.metadata.RelMetadataQuery +import org.apache.calcite.rel.{RelNode, RelWriter} +import org.apache.calcite.rex.RexProgram +import org.apache.flink.table.plan.nodes.CommonCalc +import org.apache.flink.table.plan.schema.RowSchema + +/** + * Base RelNode for data stream calc. + */ +abstract class DataStreamCalcBase( + cluster: RelOptCluster, + traitSet: RelTraitSet, + input: RelNode, + inputSchema: RowSchema, + schema: RowSchema, + calcProgram: RexProgram, + ruleDescription: String) + extends Calc(cluster, traitSet, input, calcProgram) + with CommonCalc + with DataStreamRel { + + override def deriveRowType(): RelDataType = schema.relDataType + + override def toString: String = calcToString(calcProgram, getExpressionString) + + override def explainTerms(pw: RelWriter): RelWriter = { + pw.input("input", getInput) + .item("select", selectionToString(calcProgram, getExpressionString)) + .itemIf("where", + conditionToString(calcProgram, getExpressionString), + calcProgram.getCondition != null) + } + + override def computeSelfCost(planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = { + val child = this.getInput + val rowCnt = metadata.getRowCount(child) + computeSelfCost(calcProgram, planner, rowCnt) + } + + override def estimateRowCount(metadata: RelMetadataQuery): Double = { + val child = this.getInput + val rowCnt = metadata.getRowCount(child) + estimateRowCount(calcProgram, rowCnt) + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala new file mode 100644 index 0000000000..fbb7be14bc --- /dev/null +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.plan.nodes.datastream + +import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rex.RexProgram +import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.table.api.StreamQueryConfig +import org.apache.flink.table.plan.schema.RowSchema +import org.apache.flink.table.planner.StreamPlanner +import org.apache.flink.table.runtime.types.CRow + +/** + * RelNode for Python ScalarFunctions. + */ +class DataStreamPythonCalc( + cluster: RelOptCluster, + traitSet: RelTraitSet, + input: RelNode, + inputSchema: RowSchema, + schema: RowSchema, + calcProgram: RexProgram, + ruleDescription: String) + extends DataStreamCalcBase( + cluster, + traitSet, + input, + inputSchema, + schema, + calcProgram, + ruleDescription) { + + override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = { + new DataStreamPythonCalc( + cluster, + traitSet, + child, + inputSchema, + schema, + program, + ruleDescription) + } + + override def translateToPlan( + planner: StreamPlanner, + queryConfig: StreamQueryConfig): DataStream[CRow] = { + // Will add the implementation in FLINK-14018 as it's not testable for now. + null + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala index b7701cdde0..c13ed65ea4 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala @@ -143,6 +143,13 @@ object FlinkRuleSets { FlinkLogicalWindowTableAggregate.CONVERTER ) + /** + * RuleSet to do rewrite on FlinkLogicalRel + */ + val LOGICAL_REWRITE_RULES: RuleSet = RuleSets.ofList( + PythonScalarFunctionSplitRule.INSTANCE + ) + /** * RuleSet to normalize plans for batch / DataSet execution */ @@ -233,7 +240,8 @@ object FlinkRuleSets { StreamTableSourceScanRule.INSTANCE, DataStreamMatchRule.INSTANCE, DataStreamTableAggregateRule.INSTANCE, - DataStreamGroupWindowTableAggregateRule.INSTANCE + DataStreamGroupWindowTableAggregateRule.INSTANCE, + DataStreamPythonCalcRule.INSTANCE ) /** diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala index 0a1a31a7a5..a7c16d9b84 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala @@ -18,21 +18,30 @@ package org.apache.flink.table.plan.rules.datastream -import org.apache.calcite.plan.{RelOptRule, RelTraitSet} +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet} import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.convert.ConverterRule +import org.apache.flink.table.functions.FunctionLanguage import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.nodes.datastream.DataStreamCalc import org.apache.flink.table.plan.nodes.logical.FlinkLogicalCalc import org.apache.flink.table.plan.schema.RowSchema +import org.apache.flink.table.plan.util.PythonUtil.containsFunctionOf + +import scala.collection.JavaConverters._ class DataStreamCalcRule extends ConverterRule( classOf[FlinkLogicalCalc], FlinkConventions.LOGICAL, FlinkConventions.DATASTREAM, - "DataStreamCalcRule") -{ + "DataStreamCalcRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + val program = calc.getProgram + !program.getExprList.asScala.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) + } def convert(rel: RelNode): RelNode = { val calc: FlinkLogicalCalc = rel.asInstanceOf[FlinkLogicalCalc] diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCalcRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCalcRule.scala new file mode 100644 index 0000000000..e164c09910 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCalcRule.scala @@ -0,0 +1,64 @@ +/* + * 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.plan.rules.datastream + +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.convert.ConverterRule +import org.apache.flink.table.functions.FunctionLanguage +import org.apache.flink.table.plan.nodes.FlinkConventions +import org.apache.flink.table.plan.nodes.datastream.DataStreamPythonCalc +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalCalc +import org.apache.flink.table.plan.schema.RowSchema +import org.apache.flink.table.plan.util.PythonUtil.containsFunctionOf + +import scala.collection.JavaConverters._ + +class DataStreamPythonCalcRule + extends ConverterRule( + classOf[FlinkLogicalCalc], + FlinkConventions.LOGICAL, + FlinkConventions.DATASTREAM, + "DataStreamPythonCalcRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + val program = calc.getProgram + program.getExprList.asScala.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) + } + + def convert(rel: RelNode): RelNode = { + val calc: FlinkLogicalCalc = rel.asInstanceOf[FlinkLogicalCalc] + val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM) + val convInput: RelNode = RelOptRule.convert(calc.getInput, FlinkConventions.DATASTREAM) + + new DataStreamPythonCalc( + rel.getCluster, + traitSet, + convInput, + new RowSchema(convInput.getRowType), + new RowSchema(rel.getRowType), + calc.getProgram, + description) + } +} + +object DataStreamPythonCalcRule { + val INSTANCE: RelOptRule = new DataStreamPythonCalcRule +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonScalarFunctionSplitRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonScalarFunctionSplitRule.scala new file mode 100644 index 0000000000..deb56c55de --- /dev/null +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonScalarFunctionSplitRule.scala @@ -0,0 +1,211 @@ +/* + * 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.plan.rules.logical + +import org.apache.calcite.plan.RelOptRule.{any, operand} +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} +import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode, RexProgram} +import org.apache.calcite.sql.validate.SqlValidatorUtil +import org.apache.flink.table.functions.FunctionLanguage +import org.apache.flink.table.functions.ScalarFunction +import org.apache.flink.table.functions.utils.ScalarSqlFunction +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalCalc +import org.apache.flink.table.plan.util.PythonUtil.containsFunctionOf +import org.apache.flink.table.plan.util.{InputRefVisitor, RexDefaultVisitor} + +import scala.collection.JavaConverters._ +import scala.collection.JavaConversions._ +import scala.collection.mutable + +/** + * Rule that splits [[FlinkLogicalCalc]] into multiple [[FlinkLogicalCalc]]s. After this rule + * is applied, each [[FlinkLogicalCalc]] will only contain Python [[ScalarFunction]]s or Java + * [[ScalarFunction]]s. This is to ensure that the Python [[ScalarFunction]]s which could be + * executed in a batch are grouped into the same [[FlinkLogicalCalc]] node. + */ +class PythonScalarFunctionSplitRule extends RelOptRule( + operand(classOf[FlinkLogicalCalc], any), + "PythonScalarFunctionSplitRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + val program = calc.getProgram + + // This rule matches if one of the following cases is met: + // 1. There are Python functions and Java functions mixed in the Calc + // 2. There are Python functions in the condition of the Calc + (program.getExprList.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) && + program.getExprList.exists(containsFunctionOf(_, FunctionLanguage.JVM))) || + Option(program.getCondition) + .map(program.expandLocalRef) + .exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + val input = calc.getInput + val rexBuilder = call.builder().getRexBuilder + val program = calc.getProgram + val extractedRexCalls = new mutable.ArrayBuffer[RexCall]() + + val convertPythonFunction = + program.getProjectList + .map(program.expandLocalRef) + .exists(containsFunctionOf(_, FunctionLanguage.JVM, recursive = false)) || + Option(program.getCondition) + .map(program.expandLocalRef) + .exists(expr => + containsFunctionOf(expr, FunctionLanguage.JVM, recursive = false) || + containsFunctionOf(expr, FunctionLanguage.PYTHON)) + + val extractedFunctionOffset = input.getRowType.getFieldCount + val splitter = new ScalarFunctionSplitter( + extractedFunctionOffset, + extractedRexCalls, + convertPythonFunction) + + val newProjects = program.getProjectList.map(program.expandLocalRef(_).accept(splitter)) + val newCondition = Option(program.getCondition).map(program.expandLocalRef(_).accept(splitter)) + val accessedFields = extractRefInputFields(newProjects, newCondition, extractedFunctionOffset) + + val bottomCalcProjects = + accessedFields.map(RexInputRef.of(_, input.getRowType)) ++ extractedRexCalls + val bottomCalcFieldNames = SqlValidatorUtil.uniquify( + accessedFields.map(i => input.getRowType.getFieldNames.get(i)).toSeq ++ + extractedRexCalls.indices.map("f" + _), + rexBuilder.getTypeFactory.getTypeSystem.isSchemaCaseSensitive) + + val bottomCalc = new FlinkLogicalCalc( + calc.getCluster, + calc.getTraitSet, + input, + RexProgram.create( + input.getRowType, + bottomCalcProjects.toList, + null, + bottomCalcFieldNames, + rexBuilder)) + + val inputRewriter = new ExtractedFunctionInputRewriter(extractedFunctionOffset, accessedFields) + val topCalc = new FlinkLogicalCalc( + calc.getCluster, + calc.getTraitSet, + bottomCalc, + RexProgram.create( + bottomCalc.getRowType, + newProjects.map(_.accept(inputRewriter)), + newCondition.map(_.accept(inputRewriter)).orNull, + calc.getRowType, + rexBuilder)) + + call.transformTo(topCalc) + } + + /** + * Extracts the indices of the input fields referred by the specified projects and condition. + */ + private def extractRefInputFields( + projects: Seq[RexNode], + condition: Option[RexNode], + inputFieldsCount: Int): Array[Int] = { + val visitor = new InputRefVisitor + + // extract referenced input fields from projections + projects.foreach(exp => exp.accept(visitor)) + + // extract referenced input fields from condition + condition.foreach(_.accept(visitor)) + + // fields of indexes greater than inputFieldsCount is the extracted functions and + // should be filtered as they are not from the original input + visitor.getFields.filter(_ < inputFieldsCount) + } +} + +private class ScalarFunctionSplitter( + extractedFunctionOffset: Int, + extractedRexCalls: mutable.ArrayBuffer[RexCall], + convertPythonFunction: Boolean) + extends RexDefaultVisitor[RexNode] { + + override def visitCall(call: RexCall): RexNode = { + call.getOperator match { + case sfc: ScalarSqlFunction if sfc.getScalarFunction.getLanguage == + FunctionLanguage.PYTHON => + visit(convertPythonFunction, call) + + case _ => + visit(!convertPythonFunction, call) + } + } + + override def visitNode(rexNode: RexNode): RexNode = rexNode + + private def visit(needConvert: Boolean, call: RexCall): RexNode = { + if (needConvert) { + val newNode = new RexInputRef( + extractedFunctionOffset + extractedRexCalls.length, call.getType) + extractedRexCalls.append(call) + newNode + } else { + call.clone(call.getType, call.getOperands.asScala.map(_.accept(this))) + } + } +} + +/** + * Rewrite field accesses of a RexNode as not all the fields from the original input are forwarded: + * 1) Fields of index greater than or equal to extractedFunctionOffset refer to the + * extracted function. + * 2) Fields of index less than extractedFunctionOffset refer to the original input field. + * + * @param extractedFunctionOffset the original start offset of the extracted functions + * @param accessedFields the accessed fields which will be forwarded + */ +private class ExtractedFunctionInputRewriter( + extractedFunctionOffset: Int, + accessedFields: Array[Int]) + extends RexDefaultVisitor[RexNode] { + + /** old input fields ref index -> new input fields ref index mappings */ + private val fieldMap: Map[Int, Int] = accessedFields.zipWithIndex.toMap + + override def visitInputRef(inputRef: RexInputRef): RexNode = { + if (inputRef.getIndex >= extractedFunctionOffset) { + new RexInputRef( + inputRef.getIndex - extractedFunctionOffset + accessedFields.length, + inputRef.getType) + } else { + new RexInputRef( + fieldMap.getOrElse(inputRef.getIndex, + throw new IllegalArgumentException("input field contains invalid index")), + inputRef.getType) + } + } + + override def visitCall(call: RexCall): RexNode = { + call.clone(call.getType, call.getOperands.asScala.map(_.accept(this))) + } + + override def visitNode(rexNode: RexNode): RexNode = rexNode +} + +object PythonScalarFunctionSplitRule { + val INSTANCE: RelOptRule = new PythonScalarFunctionSplitRule +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/PythonUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/PythonUtil.scala new file mode 100644 index 0000000000..cd4efd9c77 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/PythonUtil.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.plan.util + +import org.apache.calcite.rex.{RexCall, RexNode} +import org.apache.flink.table.functions.FunctionLanguage +import org.apache.flink.table.functions.utils.ScalarSqlFunction + +import scala.collection.JavaConversions._ + +object PythonUtil { + + /** + * Checks whether it contains the specified kind of function in the specified node. + * + * @param node the RexNode to check + * @param language the expected kind of function to find + * @param recursive whether check the inputs of the specified node + * @return true if it contains the specified kind of function in the specified node. + */ + def containsFunctionOf( + node: RexNode, + language: FunctionLanguage, + recursive: Boolean = true): Boolean = { + node.accept(new FunctionFinder(language, recursive)) + } + + /** + * Checks whether it contains the specified kind of function in a RexNode. + * + * @param expectedLanguage the expected kind of function to find + * @param recursive whether check the inputs + */ + class FunctionFinder(expectedLanguage: FunctionLanguage, recursive: Boolean) + extends RexDefaultVisitor[Boolean] { + + override def visitCall(call: RexCall): Boolean = { + call.getOperator match { + case sfc: ScalarSqlFunction if sfc.getScalarFunction.getLanguage == + FunctionLanguage.PYTHON => + findInternal(FunctionLanguage.PYTHON, call) + case _ => + findInternal(FunctionLanguage.JVM, call) + } + } + + override def visitNode(rexNode: RexNode): Boolean = false + + private def findInternal(actualLanguage: FunctionLanguage, call: RexCall): Boolean = + actualLanguage == expectedLanguage || + (recursive && call.getOperands.exists(_.accept(this))) + } +} diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/calcite/CalciteConfigBuilderTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/calcite/CalciteConfigBuilderTest.scala index a7ffa594a9..9dc36065ca 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/calcite/CalciteConfigBuilderTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/calcite/CalciteConfigBuilderTest.scala @@ -47,6 +47,9 @@ class CalciteConfigBuilderTest { assertFalse(cc.replacesDecoRuleSet) assertFalse(cc.decoRuleSet.isDefined) + + assertFalse(cc.replacesLogicalRewriteRuleSet) + assertFalse(cc.logicalRewriteRuleSet.isDefined) } @Test @@ -56,6 +59,7 @@ class CalciteConfigBuilderTest { .addNormRuleSet(RuleSets.ofList(ReduceExpressionsRule.FILTER_INSTANCE)) .replaceLogicalOptRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE)) .replacePhysicalOptRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE)) + .replaceLogicalRewriteRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE)) .replaceDecoRuleSet(RuleSets.ofList(DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE)) .build() @@ -65,6 +69,9 @@ class CalciteConfigBuilderTest { assertTrue(cc.replacesLogicalOptRuleSet) assertTrue(cc.logicalOptRuleSet.isDefined) + assertTrue(cc.replacesLogicalOptRuleSet) + assertTrue(cc.logicalRewriteRuleSet.isDefined) + assertTrue(cc.replacesPhysicalOptRuleSet) assertTrue(cc.physicalOptRuleSet.isDefined) @@ -182,6 +189,54 @@ class CalciteConfigBuilderTest { assertTrue(cSet.contains(CalcSplitRule.INSTANCE)) } + @Test + def testReplaceLogicalRewriteRules(): Unit = { + + val cc: CalciteConfig = new CalciteConfigBuilder() + .replaceLogicalRewriteRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE)) + .build() + + assertEquals(true, cc.replacesLogicalRewriteRuleSet) + assertTrue(cc.logicalRewriteRuleSet.isDefined) + val cSet = cc.logicalRewriteRuleSet.get.iterator().asScala.toSet + assertEquals(1, cSet.size) + assertTrue(cSet.contains(FilterMergeRule.INSTANCE)) + } + + @Test + def testReplaceLogicalRewriteAddRules(): Unit = { + + val cc: CalciteConfig = new CalciteConfigBuilder() + .replaceLogicalRewriteRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE)) + .addLogicalRewriteRuleSet(RuleSets.ofList(CalcMergeRule.INSTANCE, CalcSplitRule.INSTANCE)) + .build() + + assertEquals(true, cc.replacesLogicalRewriteRuleSet) + assertTrue(cc.logicalRewriteRuleSet.isDefined) + val cSet = cc.logicalRewriteRuleSet.get.iterator().asScala.toSet + assertEquals(3, cSet.size) + assertTrue(cSet.contains(FilterMergeRule.INSTANCE)) + assertTrue(cSet.contains(CalcMergeRule.INSTANCE)) + assertTrue(cSet.contains(CalcSplitRule.INSTANCE)) + } + + @Test + def testAddLogicalRewriteRules(): Unit = { + + val cc: CalciteConfig = new CalciteConfigBuilder() + .addLogicalRewriteRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE)) + .addLogicalRewriteRuleSet(RuleSets.ofList(CalcMergeRule.INSTANCE, CalcSplitRule.INSTANCE)) + .build() + + assertEquals(false, cc.replacesLogicalRewriteRuleSet) + assertTrue(cc.logicalRewriteRuleSet.isDefined) + val cSet = cc.logicalRewriteRuleSet.get.iterator().asScala.toSet + assertEquals(3, cSet.size) + assertTrue(cSet.contains(FilterMergeRule.INSTANCE)) + assertTrue(cSet.contains(CalcMergeRule.INSTANCE)) + assertTrue(cSet.contains(CalcSplitRule.INSTANCE)) + } + @Test def testReplacePhysicalOptimizationRules(): Unit = { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonScalarFunctionSplitRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonScalarFunctionSplitRuleTest.scala new file mode 100644 index 0000000000..3db9398733 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonScalarFunctionSplitRuleTest.scala @@ -0,0 +1,238 @@ +/* + * 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.plan + +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.functions.{FunctionLanguage, ScalarFunction} +import org.apache.flink.table.utils.TableTestUtil._ +import org.apache.flink.table.utils.TableTestBase +import org.junit.Test + +class PythonScalarFunctionSplitRuleTest extends TableTestBase { + + @Test + def testPythonFunctionAsInputOfJavaFunction(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) + + val resultTable = table + .select("pyFunc1(a, b) + 1") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "pyFunc1(a, b) AS f0") + ), + term("select", "+(f0, 1) AS _c0") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testPythonFunctionMixedWithJavaFunction(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) + + val resultTable = table + .select("pyFunc1(a, b), c + 1") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "c", "pyFunc1(a, b) AS f0") + ), + term("select", "f0 AS _c0", "+(c, 1) AS _c1") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testPythonFunctionMixedWithJavaFunctionInWhereClause(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) + util.tableEnv.registerFunction("pyFunc2", new PythonScalarFunction("pyFunc2")) + + val resultTable = table + .where("pyFunc2(a, c) > 0") + .select("pyFunc1(a, b), c + 1") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "c", "pyFunc1(a, b) AS f0", "pyFunc2(a, c) AS f1") + ), + term("select", "f0 AS _c0", "+(c, 1) AS _c1"), + term("where", ">(f1, 0)") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testPythonFunctionInWhereClause(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) + util.tableEnv.registerFunction("pyFunc2", new BooleanPythonScalarFunction("pyFunc2")) + + val resultTable = table + .where("pyFunc2(a, c)") + .select("pyFunc1(a, b)") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "pyFunc1(a, b) AS f0", "pyFunc2(a, c) AS f1") + ), + term("select", "f0 AS _c0"), + term("where", "f1") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testChainingPythonFunction(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) + util.tableEnv.registerFunction("pyFunc2", new PythonScalarFunction("pyFunc2")) + util.tableEnv.registerFunction("pyFunc3", new PythonScalarFunction("pyFunc3")) + + val resultTable = table + .select("pyFunc3(pyFunc2(a + pyFunc1(a, c), b), c)") + + val expected = unaryNode( + "DataStreamPythonCalc", + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "b", "c", "a", "pyFunc1(a, c) AS f0") + ), + term("select", "b", "c", "+(a, f0) AS f0") + ), + term("select", "pyFunc3(pyFunc2(f0, b), c) AS _c0") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testOnlyOnePythonFunction(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) + + val resultTable = table + .select("pyFunc1(a, b)") + + val expected = unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "pyFunc1(a, b) AS _c0") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testOnlyOnePythonFunctionInWhereClause(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pyFunc1", new BooleanPythonScalarFunction("pyFunc1")) + + val resultTable = table + .where("pyFunc1(a, c)") + .select("a, b") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "a", "b", "pyFunc1(a, c) AS f0") + ), + term("select", "a", "b"), + term("where", "f0") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testFieldNameUniquify(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'f0, 'f1, 'f2) + util.tableEnv.registerFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) + + val resultTable = table + .select("pyFunc1(f1, f2), f0 + 1") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "f0", "pyFunc1(f1, f2) AS f00") + ), + term("select", "f00 AS _c0", "+(f0, 1) AS _c1") + ) + + util.verifyTable(resultTable, expected) + } +} + +class PythonScalarFunction(name: String) extends ScalarFunction { + def eval(i: Int, j: Int): Int = i + j + + override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = + BasicTypeInfo.INT_TYPE_INFO + + override def getLanguage: FunctionLanguage = FunctionLanguage.PYTHON + + override def toString: String = name +} + +class BooleanPythonScalarFunction(name: String) extends ScalarFunction { + def eval(i: Int, j: Int): Boolean = i + j > 1 + + override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = + BasicTypeInfo.BOOLEAN_TYPE_INFO + + override def getLanguage: FunctionLanguage = FunctionLanguage.PYTHON + + override def toString: String = name +} -- Gitee From b65370bd89a90f3909eef37c2462012bc98dd10d Mon Sep 17 00:00:00 2001 From: vinoyang Date: Fri, 27 Sep 2019 14:28:47 +0800 Subject: [PATCH 050/268] [FLINK-13143][checkpointing] Remove unnecessary CheckpointExceptionHandler class (#9777) --- .../tasks/CheckpointExceptionHandler.java | 36 --------- .../CheckpointExceptionHandlerFactory.java | 59 -------------- .../streaming/runtime/tasks/StreamTask.java | 18 +---- .../tasks/CheckpointExceptionHandlerTest.java | 77 ------------------- .../runtime/tasks/StreamTaskTest.java | 28 ++++++- 5 files changed, 30 insertions(+), 188 deletions(-) delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandler.java delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerFactory.java delete mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerTest.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandler.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandler.java deleted file mode 100644 index 0140795255..0000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandler.java +++ /dev/null @@ -1,36 +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.runtime.tasks; - -import org.apache.flink.runtime.checkpoint.CheckpointMetaData; - -/** - * Handler for exceptions that happen on checkpointing. The handler can reject and rethrow the offered exceptions. - */ -public interface CheckpointExceptionHandler { - - /** - * Offers the exception for handling. If the exception cannot be handled from this instance, it is rethrown. - * - * @param checkpointMetaData metadata for the checkpoint for which the exception occurred. - * @param exception the exception to handle. - * @throws Exception rethrows the exception if it cannot be handled. - */ - void tryHandleCheckpointException(CheckpointMetaData checkpointMetaData, Exception exception) throws Exception; -} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerFactory.java deleted file mode 100644 index 64ab71ce00..0000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerFactory.java +++ /dev/null @@ -1,59 +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.runtime.tasks; - -import org.apache.flink.runtime.checkpoint.CheckpointMetaData; -import org.apache.flink.runtime.execution.Environment; -import org.apache.flink.util.Preconditions; - -/** - * This factory produces {@link CheckpointExceptionHandler} instances that handle exceptions during checkpointing in a - * {@link StreamTask}. - */ -public class CheckpointExceptionHandlerFactory { - - /** - * Returns a {@link CheckpointExceptionHandler} that just declines checkpoint on exception. - */ - public CheckpointExceptionHandler createCheckpointExceptionHandler( - Environment environment) { - - return new DecliningCheckpointExceptionHandler(environment); - } - - /** - * This handler makes the task decline the checkpoint as reaction to the reported exception. The task is not failed. - */ - static final class DecliningCheckpointExceptionHandler implements CheckpointExceptionHandler { - - final Environment environment; - - DecliningCheckpointExceptionHandler(Environment environment) { - this.environment = Preconditions.checkNotNull(environment); - } - - @Override - public void tryHandleCheckpointException( - CheckpointMetaData checkpointMetaData, - Exception exception) { - - environment.declineCheckpoint(checkpointMetaData.getCheckpointId(), exception); - } - } -} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 148a43d3c8..1617ac7fe2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -200,9 +200,6 @@ public abstract class StreamTask> /** Thread pool for async snapshot workers. */ private ExecutorService asyncOperationsThreadPool; - /** Handler for exceptions during checkpointing in the stream task. Used in synchronous part of the checkpoint. */ - private CheckpointExceptionHandler checkpointExceptionHandler; - private final List>>> recordWriters; protected final MailboxProcessor mailboxProcessor; @@ -375,11 +372,6 @@ public abstract class StreamTask> asyncOperationsThreadPool = Executors.newCachedThreadPool(new ExecutorThreadFactory("AsyncOperations", uncaughtExceptionHandler)); - CheckpointExceptionHandlerFactory cpExceptionHandlerFactory = createCheckpointExceptionHandlerFactory(); - - checkpointExceptionHandler = cpExceptionHandlerFactory - .createCheckpointExceptionHandler(getEnvironment()); - stateBackend = createStateBackend(); checkpointStorage = stateBackend.createCheckpointStorage(getEnvironment().getJobID()); @@ -974,10 +966,6 @@ public abstract class StreamTask> LOG); } - protected CheckpointExceptionHandlerFactory createCheckpointExceptionHandlerFactory() { - return new CheckpointExceptionHandlerFactory(); - } - /** * Returns the {@link ProcessingTimeService} responsible for telling the current * processing time and registering timers. @@ -1186,7 +1174,7 @@ public abstract class StreamTask> // We only report the exception for the original cause of fail and cleanup. // Otherwise this followup exception could race the original exception in failing the task. try { - owner.checkpointExceptionHandler.tryHandleCheckpointException(checkpointMetaData, checkpointException); + owner.getEnvironment().declineCheckpoint(checkpointMetaData.getCheckpointId(), checkpointException); } catch (Exception unhandled) { AsynchronousException asyncException = new AsynchronousException(unhandled); owner.handleAsyncException("Failure in asynchronous checkpoint materialization", asyncException); @@ -1348,12 +1336,12 @@ public abstract class StreamTask> if (checkpointOptions.getCheckpointType().isSynchronous()) { // in the case of a synchronous checkpoint, we always rethrow the exception, - // so that the task fails (as if we had the FailingCheckpointExceptionHandler). + // so that the task fails. // this is because the intention is always to stop the job after this checkpointing // operation, and without the failure, the task would go back to normal execution. throw ex; } else { - owner.checkpointExceptionHandler.tryHandleCheckpointException(checkpointMetaData, ex); + owner.getEnvironment().declineCheckpoint(checkpointMetaData.getCheckpointId(), ex); } } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerTest.java deleted file mode 100644 index 2632c01027..0000000000 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CheckpointExceptionHandlerTest.java +++ /dev/null @@ -1,77 +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.runtime.tasks; - -import org.apache.flink.runtime.checkpoint.CheckpointMetaData; -import org.apache.flink.runtime.operators.testutils.DummyEnvironment; -import org.apache.flink.util.TestLogger; - -import org.junit.Assert; -import org.junit.Test; - -/** - * Test for the current implementations of {@link CheckpointExceptionHandler} and their factory. - */ -public class CheckpointExceptionHandlerTest extends TestLogger { - - @Test - public void testDecliningHandler() { - DeclineDummyEnvironment environment = new DeclineDummyEnvironment(); - CheckpointExceptionHandlerFactory checkpointExceptionHandlerFactory = new CheckpointExceptionHandlerFactory(); - CheckpointExceptionHandler exceptionHandler = - checkpointExceptionHandlerFactory.createCheckpointExceptionHandler(environment); - - CheckpointMetaData failedCheckpointMetaData = new CheckpointMetaData(42L, 4711L); - Exception testException = new Exception("test"); - try { - exceptionHandler.tryHandleCheckpointException(failedCheckpointMetaData, testException); - } catch (Exception e) { - Assert.fail("Exception not handled, but rethrown."); - } - - Assert.assertEquals(failedCheckpointMetaData.getCheckpointId(), environment.getLastDeclinedCheckpointId()); - Assert.assertEquals(testException, environment.getLastDeclinedCheckpointCause()); - } - - static final class DeclineDummyEnvironment extends DummyEnvironment { - - private long lastDeclinedCheckpointId; - private Throwable lastDeclinedCheckpointCause; - - DeclineDummyEnvironment() { - super("test", 1, 0); - this.lastDeclinedCheckpointId = Long.MIN_VALUE; - this.lastDeclinedCheckpointCause = null; - } - - @Override - public void declineCheckpoint(long checkpointId, Throwable cause) { - this.lastDeclinedCheckpointId = checkpointId; - this.lastDeclinedCheckpointCause = cause; - } - - long getLastDeclinedCheckpointId() { - return lastDeclinedCheckpointId; - } - - Throwable getLastDeclinedCheckpointCause() { - return lastDeclinedCheckpointCause; - } - } -} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 23b1a46258..297c0640dc 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -348,7 +348,7 @@ public class StreamTaskTest extends TestLogger { @Test public void testDecliningCheckpointStreamOperator() throws Exception { - CheckpointExceptionHandlerTest.DeclineDummyEnvironment declineDummyEnvironment = new CheckpointExceptionHandlerTest.DeclineDummyEnvironment(); + DeclineDummyEnvironment declineDummyEnvironment = new DeclineDummyEnvironment(); // mock the returned snapshots OperatorSnapshotFutures operatorSnapshotResult1 = mock(OperatorSnapshotFutures.class); @@ -1669,4 +1669,30 @@ public class StreamTaskTest extends TestLogger { throw failingCause; } } + + static final class DeclineDummyEnvironment extends DummyEnvironment { + + private long lastDeclinedCheckpointId; + private Throwable lastDeclinedCheckpointCause; + + DeclineDummyEnvironment() { + super("test", 1, 0); + this.lastDeclinedCheckpointId = Long.MIN_VALUE; + this.lastDeclinedCheckpointCause = null; + } + + @Override + public void declineCheckpoint(long checkpointId, Throwable cause) { + this.lastDeclinedCheckpointId = checkpointId; + this.lastDeclinedCheckpointCause = cause; + } + + long getLastDeclinedCheckpointId() { + return lastDeclinedCheckpointId; + } + + Throwable getLastDeclinedCheckpointCause() { + return lastDeclinedCheckpointCause; + } + } } -- Gitee From b70639d372f75c9bf602d096ef691b6d7bd4b607 Mon Sep 17 00:00:00 2001 From: Gao Yun Date: Thu, 26 Sep 2019 23:35:33 +0800 Subject: [PATCH 051/268] [FLINK-14186][e2e] Skip Elastic Search 2.3.5 case when running with JDK11 This closes #9775. --- tools/travis/splits/split_misc.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tools/travis/splits/split_misc.sh b/tools/travis/splits/split_misc.sh index 856b351bf4..7a4c8a06f7 100755 --- a/tools/travis/splits/split_misc.sh +++ b/tools/travis/splits/split_misc.sh @@ -59,7 +59,9 @@ if [[ ${PROFILE} != *"jdk11"* ]]; then fi 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 (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" +if [[ ${PROFILE} != *"jdk11"* ]]; then + 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" +fi 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" -- Gitee From 72ea59e7e4eac07a8c5d28661d0a329c758193c8 Mon Sep 17 00:00:00 2001 From: "kevin.cyj" Date: Wed, 25 Sep 2019 17:06:16 +0800 Subject: [PATCH 052/268] [FLINK-14195] Add jaxb dependency and do relocation for java 11 in s3. --- flink-filesystems/flink-s3-fs-hadoop/pom.xml | 43 ++++++++++++++++++++ flink-filesystems/flink-s3-fs-presto/pom.xml | 43 ++++++++++++++++++++ 2 files changed, 86 insertions(+) diff --git a/flink-filesystems/flink-s3-fs-hadoop/pom.xml b/flink-filesystems/flink-s3-fs-hadoop/pom.xml index e03c885f34..d04def170f 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/pom.xml +++ b/flink-filesystems/flink-s3-fs-hadoop/pom.xml @@ -126,4 +126,47 @@ under the License. + + + + java11 + + 11 + + + + + javax.xml.bind + jaxb-api + 2.3.0 + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + javax.xml.bind + org.apache.flink.fs.s3hadoop.shaded.javax.xml.bind + + + + + + + + + + + diff --git a/flink-filesystems/flink-s3-fs-presto/pom.xml b/flink-filesystems/flink-s3-fs-presto/pom.xml index f589e04884..b56c77ed4c 100644 --- a/flink-filesystems/flink-s3-fs-presto/pom.xml +++ b/flink-filesystems/flink-s3-fs-presto/pom.xml @@ -323,4 +323,47 @@ under the License. + + + + java11 + + 11 + + + + + javax.xml.bind + jaxb-api + 2.3.0 + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + javax.xml.bind + org.apache.flink.fs.s3presto.shaded.javax.xml.bind + + + + + + + + + + + -- Gitee From 249cae1ce746a4a639cfc3b83f38f3c5b34e3dba Mon Sep 17 00:00:00 2001 From: "kevin.cyj" Date: Wed, 25 Sep 2019 17:11:34 +0800 Subject: [PATCH 053/268] [FLINK-14195] Add Streaming File Sink s3 end-to-end test back for java 11. --- tools/travis/splits/split_misc.sh | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tools/travis/splits/split_misc.sh b/tools/travis/splits/split_misc.sh index 7a4c8a06f7..73fa7682ea 100755 --- a/tools/travis/splits/split_misc.sh +++ b/tools/travis/splits/split_misc.sh @@ -54,9 +54,7 @@ run_test "Streaming SQL end-to-end test (Old planner)" "$END_TO_END_DIR/test-scr run_test "Streaming SQL end-to-end test (Blink planner)" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh blink" "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" -if [[ ${PROFILE} != *"jdk11"* ]]; then - 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" -fi +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" if [[ ${PROFILE} != *"jdk11"* ]]; then -- Gitee From 96047a25d29c53106170e2be376be198c5a6b5c2 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Mon, 9 Sep 2019 14:29:01 +0200 Subject: [PATCH 054/268] [hotfix][runtime] Make slotSharingGroupId nullable in TestingLogicalSlot --- .../apache/flink/runtime/jobmaster/TestingLogicalSlot.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingLogicalSlot.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingLogicalSlot.java index 3304c96189..de67e416eb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingLogicalSlot.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingLogicalSlot.java @@ -53,6 +53,7 @@ public class TestingLogicalSlot implements LogicalSlot { private final SlotRequestId slotRequestId; + @Nullable private final SlotSharingGroupId slotSharingGroupId; TestingLogicalSlot( @@ -61,7 +62,7 @@ public class TestingLogicalSlot implements LogicalSlot { int slotNumber, AllocationID allocationId, SlotRequestId slotRequestId, - SlotSharingGroupId slotSharingGroupId, + @Nullable SlotSharingGroupId slotSharingGroupId, boolean automaticallyCompleteReleaseFuture, SlotOwner slotOwner) { @@ -71,7 +72,7 @@ public class TestingLogicalSlot implements LogicalSlot { this.slotNumber = slotNumber; this.allocationId = Preconditions.checkNotNull(allocationId); this.slotRequestId = Preconditions.checkNotNull(slotRequestId); - this.slotSharingGroupId = Preconditions.checkNotNull(slotSharingGroupId); + this.slotSharingGroupId = slotSharingGroupId; this.releaseFuture = new CompletableFuture<>(); this.automaticallyCompleteReleaseFuture = automaticallyCompleteReleaseFuture; this.slotOwner = slotOwner; -- Gitee From e361fcb310705159decab2708063f80cd6d55f5c Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Mon, 9 Sep 2019 11:15:20 +0200 Subject: [PATCH 055/268] [hotfix][runtime] Set correct SlotSharingGroupId in LogicalSlot returned by SimpleSlotProvider Set SlotSharingGroupId to the Task's SlotSharingGroupId in TestingLogicalSlot returned by SimpleSlotProvider --- .../flink/runtime/executiongraph/utils/SimpleSlotProvider.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java index 15ec34379a..f0db357aca 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java @@ -106,6 +106,7 @@ public class SimpleSlotProvider implements SlotProvider, SlotOwner { .setSlotNumber(slot.getPhysicalSlotNumber()) .setAllocationId(slot.getAllocationId()) .setSlotRequestId(slotRequestId) + .setSlotSharingGroupId(task.getSlotSharingGroupId()) .setSlotOwner(this) .createTestingLogicalSlot(); allocatedSlots.put(slotRequestId, slot); -- Gitee From 83719f9d998c3b57b755ea75c682a7e3bc54764a Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Thu, 12 Sep 2019 10:16:31 +0200 Subject: [PATCH 056/268] [hotfix][runtime] Add checkNotNull for slotOwner ctor parameter in TestingLogicalSlot --- .../org/apache/flink/runtime/jobmaster/TestingLogicalSlot.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingLogicalSlot.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingLogicalSlot.java index de67e416eb..ff680e36c7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingLogicalSlot.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingLogicalSlot.java @@ -75,7 +75,7 @@ public class TestingLogicalSlot implements LogicalSlot { this.slotSharingGroupId = slotSharingGroupId; this.releaseFuture = new CompletableFuture<>(); this.automaticallyCompleteReleaseFuture = automaticallyCompleteReleaseFuture; - this.slotOwner = slotOwner; + this.slotOwner = Preconditions.checkNotNull(slotOwner); } @Override -- Gitee From 064209650fc695f3d3e4a26a6bbd9785167eb840 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Mon, 9 Sep 2019 10:44:56 +0200 Subject: [PATCH 057/268] [FLINK-12433][runtime] Add NoOpFailoverStrategy Introduce a NoOpFailoverStrategy that is exclusively configured if the new generation scheduler is used. This is because the new scheduler does not depend on the legacy FailoverStrategy interface. --- .../failover/FailoverStrategyLoader.java | 12 ++++- .../failover/NoOpFailoverStrategy.java | 53 +++++++++++++++++++ 2 files changed, 64 insertions(+), 1 deletion(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/NoOpFailoverStrategy.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/FailoverStrategyLoader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/FailoverStrategyLoader.java index e80c6edf2a..632f3af479 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/FailoverStrategyLoader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/FailoverStrategyLoader.java @@ -44,13 +44,20 @@ public class FailoverStrategyLoader { /** Config name for the {@link RestartPipelinedRegionStrategy}. */ public static final String LEGACY_PIPELINED_REGION_RESTART_STRATEGY_NAME = "region-legacy"; + /** Config name for the {@link NoOpFailoverStrategy}. */ + public static final String NO_OP_FAILOVER_STRATEGY = "noop"; + // ------------------------------------------------------------------------ /** * Loads a FailoverStrategy Factory from the given configuration. */ public static FailoverStrategy.Factory loadFailoverStrategy(Configuration config, @Nullable Logger logger) { - final String strategyParam = config.getString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY); + // The new generation scheduler does not depend on the FailoverStrategy loaded here. + // Therefore, we load a noop failover strategy if the new generation scheduler is configured. + final String strategyParam = config.getString(JobManagerOptions.SCHEDULER).equals("ng") ? + NO_OP_FAILOVER_STRATEGY : + config.getString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY); if (StringUtils.isNullOrWhitespaceOnly(strategyParam)) { if (logger != null) { @@ -74,6 +81,9 @@ public class FailoverStrategyLoader { case INDIVIDUAL_RESTART_STRATEGY_NAME: return new RestartIndividualStrategy.Factory(); + case NO_OP_FAILOVER_STRATEGY: + return new NoOpFailoverStrategy.Factory(); + default: // we could interpret the parameter as a factory class name and instantiate that // for now we simply do not support this diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/NoOpFailoverStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/NoOpFailoverStrategy.java new file mode 100644 index 0000000000..0cc37808eb --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/NoOpFailoverStrategy.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.runtime.executiongraph.failover; + +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; + +import java.util.List; + +/** + * FailoverStrategy that does not do anything. + */ +public class NoOpFailoverStrategy extends FailoverStrategy { + + @Override + public void onTaskFailure(final Execution taskExecution, final Throwable cause) { + } + + @Override + public void notifyNewVertices(final List newJobVerticesTopological) { + } + + @Override + public String getStrategyName() { + return "NoOp failover strategy"; + } + + public static class Factory implements FailoverStrategy.Factory { + + @Override + public FailoverStrategy create(final ExecutionGraph executionGraph) { + return new NoOpFailoverStrategy(); + } + } +} -- Gitee From e865c260da57ed67b111744d427825002e1a47e2 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Thu, 5 Sep 2019 09:34:50 +0200 Subject: [PATCH 058/268] [FLINK-12433][runtime] Introduce SchedulerBase Introduce common super class SchedulerBase for DefaultScheduler and LegacyScheduler, which contains code that is shared between the scheduler implementations. Previously, DefaultScheduler inherited from LegacyScheduler to avoid re-implementing features such as queryable state, taking savepoints, etc. --- .../runtime/scheduler/DefaultScheduler.java | 2 +- .../runtime/scheduler/LegacyScheduler.java | 524 +------------- .../runtime/scheduler/SchedulerBase.java | 644 ++++++++++++++++++ 3 files changed, 653 insertions(+), 517 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java index 05429ac701..759a56a2f4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java @@ -39,7 +39,7 @@ import java.util.concurrent.ScheduledExecutorService; /** * Stub implementation of the future default scheduler. */ -public class DefaultScheduler extends LegacyScheduler { +public class DefaultScheduler extends SchedulerBase { public DefaultScheduler( final Logger log, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/LegacyScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/LegacyScheduler.java index e33b6b6779..bc6ed69be0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/LegacyScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/LegacyScheduler.java @@ -96,39 +96,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; * * @see ExecutionGraph#scheduleForExecution() */ -public class LegacyScheduler implements SchedulerNG { - - private final Logger log; - - private final JobGraph jobGraph; - - private final ExecutionGraph executionGraph; - - private final BackPressureStatsTracker backPressureStatsTracker; - - private final Executor ioExecutor; - - private final Configuration jobMasterConfiguration; - - private final SlotProvider slotProvider; - - private final ScheduledExecutorService futureExecutor; - - private final ClassLoader userCodeLoader; - - private final CheckpointRecoveryFactory checkpointRecoveryFactory; - - private final Time rpcTimeout; - - private final RestartStrategy restartStrategy; - - private final BlobWriter blobWriter; - - private final Time slotRequestTimeout; - - private ComponentMainThreadExecutor mainThreadExecutor = new ComponentMainThreadExecutor.DummyComponentMainThreadExecutor( - "LegacyScheduler is not initialized with proper main thread executor. " + - "Call to LegacyScheduler.setMainThreadExecutor(...) required."); +public class LegacyScheduler extends SchedulerBase { public LegacyScheduler( final Logger log, @@ -148,498 +116,22 @@ public class LegacyScheduler implements SchedulerNG { final ShuffleMaster shuffleMaster, final PartitionTracker partitionTracker) throws Exception { - this.log = checkNotNull(log); - this.jobGraph = checkNotNull(jobGraph); - this.backPressureStatsTracker = checkNotNull(backPressureStatsTracker); - this.ioExecutor = checkNotNull(ioExecutor); - this.jobMasterConfiguration = checkNotNull(jobMasterConfiguration); - this.slotProvider = checkNotNull(slotProvider); - this.futureExecutor = checkNotNull(futureExecutor); - this.userCodeLoader = checkNotNull(userCodeLoader); - this.checkpointRecoveryFactory = checkNotNull(checkpointRecoveryFactory); - this.rpcTimeout = checkNotNull(rpcTimeout); - - final RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration = - jobGraph.getSerializedExecutionConfig() - .deserializeValue(userCodeLoader) - .getRestartStrategy(); - - this.restartStrategy = RestartStrategyResolving.resolve(restartStrategyConfiguration, - restartStrategyFactory, - jobGraph.isCheckpointingEnabled()); - - log.info("Using restart strategy {} for {} ({}).", this.restartStrategy, jobGraph.getName(), jobGraph.getJobID()); - - this.blobWriter = checkNotNull(blobWriter); - this.slotRequestTimeout = checkNotNull(slotRequestTimeout); - - this.executionGraph = createAndRestoreExecutionGraph(jobManagerJobMetricGroup, checkNotNull(shuffleMaster), checkNotNull(partitionTracker)); - } - - private ExecutionGraph createAndRestoreExecutionGraph( - JobManagerJobMetricGroup currentJobManagerJobMetricGroup, - ShuffleMaster shuffleMaster, - PartitionTracker partitionTracker) throws Exception { - - ExecutionGraph newExecutionGraph = createExecutionGraph(currentJobManagerJobMetricGroup, shuffleMaster, partitionTracker); - - final CheckpointCoordinator checkpointCoordinator = newExecutionGraph.getCheckpointCoordinator(); - - if (checkpointCoordinator != null) { - // check whether we find a valid checkpoint - if (!checkpointCoordinator.restoreLatestCheckpointedState( - newExecutionGraph.getAllVertices(), - false, - false)) { - - // check whether we can restore from a savepoint - tryRestoreExecutionGraphFromSavepoint(newExecutionGraph, jobGraph.getSavepointRestoreSettings()); - } - } - - return newExecutionGraph; - } - - private ExecutionGraph createExecutionGraph( - JobManagerJobMetricGroup currentJobManagerJobMetricGroup, - ShuffleMaster shuffleMaster, - final PartitionTracker partitionTracker) throws JobExecutionException, JobException { - return ExecutionGraphBuilder.buildGraph( - null, + super( + log, jobGraph, - jobMasterConfiguration, - futureExecutor, + backPressureStatsTracker, ioExecutor, + jobMasterConfiguration, slotProvider, + futureExecutor, userCodeLoader, checkpointRecoveryFactory, rpcTimeout, - restartStrategy, - currentJobManagerJobMetricGroup, + restartStrategyFactory, blobWriter, + jobManagerJobMetricGroup, slotRequestTimeout, - log, shuffleMaster, partitionTracker); } - - /** - * Tries to restore the given {@link ExecutionGraph} from the provided {@link SavepointRestoreSettings}. - * - * @param executionGraphToRestore {@link ExecutionGraph} which is supposed to be restored - * @param savepointRestoreSettings {@link SavepointRestoreSettings} containing information about the savepoint to restore from - * @throws Exception if the {@link ExecutionGraph} could not be restored - */ - private void tryRestoreExecutionGraphFromSavepoint(ExecutionGraph executionGraphToRestore, SavepointRestoreSettings savepointRestoreSettings) throws Exception { - if (savepointRestoreSettings.restoreSavepoint()) { - final CheckpointCoordinator checkpointCoordinator = executionGraphToRestore.getCheckpointCoordinator(); - if (checkpointCoordinator != null) { - checkpointCoordinator.restoreSavepoint( - savepointRestoreSettings.getRestorePath(), - savepointRestoreSettings.allowNonRestoredState(), - executionGraphToRestore.getAllVertices(), - userCodeLoader); - } - } - } - - @Override - public void setMainThreadExecutor(final ComponentMainThreadExecutor mainThreadExecutor) { - this.mainThreadExecutor = checkNotNull(mainThreadExecutor); - executionGraph.start(mainThreadExecutor); - } - - @Override - public void registerJobStatusListener(final JobStatusListener jobStatusListener) { - executionGraph.registerJobStatusListener(jobStatusListener); - } - - @Override - public void startScheduling() { - mainThreadExecutor.assertRunningInMainThread(); - - try { - executionGraph.scheduleForExecution(); - } - catch (Throwable t) { - executionGraph.failGlobal(t); - } - } - - @Override - public void suspend(Throwable cause) { - mainThreadExecutor.assertRunningInMainThread(); - executionGraph.suspend(cause); - } - - @Override - public void cancel() { - mainThreadExecutor.assertRunningInMainThread(); - executionGraph.cancel(); - } - - @Override - public CompletableFuture getTerminationFuture() { - return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn()); - } - - @Override - public boolean updateTaskExecutionState(final TaskExecutionState taskExecutionState) { - mainThreadExecutor.assertRunningInMainThread(); - return executionGraph.updateState(taskExecutionState); - } - - @Override - public SerializedInputSplit requestNextInputSplit(JobVertexID vertexID, ExecutionAttemptID executionAttempt) throws IOException { - mainThreadExecutor.assertRunningInMainThread(); - - final Execution execution = executionGraph.getRegisteredExecutions().get(executionAttempt); - if (execution == null) { - // can happen when JobManager had already unregistered this execution upon on task failure, - // but TaskManager get some delay to aware of that situation - if (log.isDebugEnabled()) { - log.debug("Can not find Execution for attempt {}.", executionAttempt); - } - // but we should TaskManager be aware of this - throw new IllegalArgumentException("Can not find Execution for attempt " + executionAttempt); - } - - final ExecutionJobVertex vertex = executionGraph.getJobVertex(vertexID); - if (vertex == null) { - throw new IllegalArgumentException("Cannot find execution vertex for vertex ID " + vertexID); - } - - if (vertex.getSplitAssigner() == null) { - throw new IllegalStateException("No InputSplitAssigner for vertex ID " + vertexID); - } - - final InputSplit nextInputSplit = execution.getNextInputSplit(); - - if (log.isDebugEnabled()) { - log.debug("Send next input split {}.", nextInputSplit); - } - - try { - final byte[] serializedInputSplit = InstantiationUtil.serializeObject(nextInputSplit); - return new SerializedInputSplit(serializedInputSplit); - } catch (Exception ex) { - IOException reason = new IOException("Could not serialize the next input split of class " + - nextInputSplit.getClass() + ".", ex); - vertex.fail(reason); - throw reason; - } - } - - @Override - public ExecutionState requestPartitionState( - final IntermediateDataSetID intermediateResultId, - final ResultPartitionID resultPartitionId) throws PartitionProducerDisposedException { - - mainThreadExecutor.assertRunningInMainThread(); - - final Execution execution = executionGraph.getRegisteredExecutions().get(resultPartitionId.getProducerId()); - if (execution != null) { - return execution.getState(); - } - else { - final IntermediateResult intermediateResult = - executionGraph.getAllIntermediateResults().get(intermediateResultId); - - if (intermediateResult != null) { - // Try to find the producing execution - Execution producerExecution = intermediateResult - .getPartitionById(resultPartitionId.getPartitionId()) - .getProducer() - .getCurrentExecutionAttempt(); - - if (producerExecution.getAttemptId().equals(resultPartitionId.getProducerId())) { - return producerExecution.getState(); - } else { - throw new PartitionProducerDisposedException(resultPartitionId); - } - } else { - throw new IllegalArgumentException("Intermediate data set with ID " - + intermediateResultId + " not found."); - } - } - } - - @Override - public void scheduleOrUpdateConsumers(final ResultPartitionID partitionID) { - mainThreadExecutor.assertRunningInMainThread(); - - try { - executionGraph.scheduleOrUpdateConsumers(partitionID); - } catch (ExecutionGraphException e) { - throw new RuntimeException(e); - } - } - - @Override - public ArchivedExecutionGraph requestJob() { - mainThreadExecutor.assertRunningInMainThread(); - return ArchivedExecutionGraph.createFrom(executionGraph); - } - - @Override - public JobStatus requestJobStatus() { - return executionGraph.getState(); - } - - @Override - public JobDetails requestJobDetails() { - mainThreadExecutor.assertRunningInMainThread(); - return WebMonitorUtils.createDetailsForJob(executionGraph); - } - - @Override - public KvStateLocation requestKvStateLocation(final JobID jobId, final String registrationName) throws UnknownKvStateLocation, FlinkJobNotFoundException { - mainThreadExecutor.assertRunningInMainThread(); - - // sanity check for the correct JobID - if (jobGraph.getJobID().equals(jobId)) { - if (log.isDebugEnabled()) { - log.debug("Lookup key-value state for job {} with registration " + - "name {}.", jobGraph.getJobID(), registrationName); - } - - final KvStateLocationRegistry registry = executionGraph.getKvStateLocationRegistry(); - final KvStateLocation location = registry.getKvStateLocation(registrationName); - if (location != null) { - return location; - } else { - throw new UnknownKvStateLocation(registrationName); - } - } else { - if (log.isDebugEnabled()) { - log.debug("Request of key-value state location for unknown job {} received.", jobId); - } - throw new FlinkJobNotFoundException(jobId); - } - } - - @Override - public void notifyKvStateRegistered(final JobID jobId, final JobVertexID jobVertexId, final KeyGroupRange keyGroupRange, final String registrationName, final KvStateID kvStateId, final InetSocketAddress kvStateServerAddress) throws FlinkJobNotFoundException { - mainThreadExecutor.assertRunningInMainThread(); - - if (jobGraph.getJobID().equals(jobId)) { - if (log.isDebugEnabled()) { - log.debug("Key value state registered for job {} under name {}.", - jobGraph.getJobID(), registrationName); - } - - try { - executionGraph.getKvStateLocationRegistry().notifyKvStateRegistered( - jobVertexId, keyGroupRange, registrationName, kvStateId, kvStateServerAddress); - } catch (Exception e) { - throw new RuntimeException(e); - } - } else { - throw new FlinkJobNotFoundException(jobId); - } - } - - @Override - public void notifyKvStateUnregistered(final JobID jobId, final JobVertexID jobVertexId, final KeyGroupRange keyGroupRange, final String registrationName) throws FlinkJobNotFoundException { - mainThreadExecutor.assertRunningInMainThread(); - - if (jobGraph.getJobID().equals(jobId)) { - if (log.isDebugEnabled()) { - log.debug("Key value state unregistered for job {} under name {}.", - jobGraph.getJobID(), registrationName); - } - - try { - executionGraph.getKvStateLocationRegistry().notifyKvStateUnregistered( - jobVertexId, keyGroupRange, registrationName); - } catch (Exception e) { - throw new RuntimeException(e); - } - } else { - throw new FlinkJobNotFoundException(jobId); - } - } - - @Override - public void updateAccumulators(final AccumulatorSnapshot accumulatorSnapshot) { - mainThreadExecutor.assertRunningInMainThread(); - - executionGraph.updateAccumulators(accumulatorSnapshot); - } - - @Override - public Optional requestOperatorBackPressureStats(final JobVertexID jobVertexId) throws FlinkException { - final ExecutionJobVertex jobVertex = executionGraph.getJobVertex(jobVertexId); - if (jobVertex == null) { - throw new FlinkException("JobVertexID not found " + - jobVertexId); - } - - return backPressureStatsTracker.getOperatorBackPressureStats(jobVertex); - } - - @Override - public CompletableFuture triggerSavepoint(final String targetDirectory, final boolean cancelJob) { - mainThreadExecutor.assertRunningInMainThread(); - - final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); - if (checkpointCoordinator == null) { - throw new IllegalStateException( - String.format("Job %s is not a streaming job.", jobGraph.getJobID())); - } else if (targetDirectory == null && !checkpointCoordinator.getCheckpointStorage().hasDefaultSavepointLocation()) { - log.info("Trying to cancel job {} with savepoint, but no savepoint directory configured.", jobGraph.getJobID()); - - throw new IllegalStateException( - "No savepoint directory configured. You can either specify a directory " + - "while cancelling via -s :targetDirectory or configure a cluster-wide " + - "default via key '" + CheckpointingOptions.SAVEPOINT_DIRECTORY.key() + "'."); - } - - if (cancelJob) { - checkpointCoordinator.stopCheckpointScheduler(); - } - - return checkpointCoordinator - .triggerSavepoint(System.currentTimeMillis(), targetDirectory) - .thenApply(CompletedCheckpoint::getExternalPointer) - .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(); - } - return path; - }, mainThreadExecutor); - } - - private void startCheckpointScheduler(final CheckpointCoordinator checkpointCoordinator) { - mainThreadExecutor.assertRunningInMainThread(); - - if (checkpointCoordinator.isPeriodicCheckpointingConfigured()) { - try { - checkpointCoordinator.startCheckpointScheduler(); - } catch (IllegalStateException ignored) { - // Concurrent shut down of the coordinator - } - } - } - - @Override - public void acknowledgeCheckpoint(final JobID jobID, final ExecutionAttemptID executionAttemptID, final long checkpointId, final CheckpointMetrics checkpointMetrics, final TaskStateSnapshot checkpointState) { - mainThreadExecutor.assertRunningInMainThread(); - - final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); - final AcknowledgeCheckpoint ackMessage = new AcknowledgeCheckpoint( - jobID, - executionAttemptID, - checkpointId, - checkpointMetrics, - checkpointState); - - final String taskManagerLocationInfo = retrieveTaskManagerLocation(executionAttemptID); - - if (checkpointCoordinator != null) { - ioExecutor.execute(() -> { - try { - checkpointCoordinator.receiveAcknowledgeMessage(ackMessage, taskManagerLocationInfo); - } catch (Throwable t) { - log.warn("Error while processing checkpoint acknowledgement message", t); - } - }); - } else { - 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()); - } - } - } - - @Override - public void declineCheckpoint(final DeclineCheckpoint decline) { - mainThreadExecutor.assertRunningInMainThread(); - - final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); - final String taskManagerLocationInfo = retrieveTaskManagerLocation(decline.getTaskExecutionId()); - - if (checkpointCoordinator != null) { - ioExecutor.execute(() -> { - try { - checkpointCoordinator.receiveDeclineMessage(decline, taskManagerLocationInfo); - } catch (Exception e) { - log.error("Error in CheckpointCoordinator while processing {}", decline, e); - } - }); - } else { - 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()); - } - } - } - - @Override - public CompletableFuture stopWithSavepoint(final String targetDirectory, final boolean advanceToEndOfEventTime) { - mainThreadExecutor.assertRunningInMainThread(); - - final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); - - if (checkpointCoordinator == null) { - return FutureUtils.completedExceptionally(new IllegalStateException( - String.format("Job %s is not a streaming job.", jobGraph.getJobID()))); - } - - if (targetDirectory == null && !checkpointCoordinator.getCheckpointStorage().hasDefaultSavepointLocation()) { - log.info("Trying to cancel job {} with savepoint, but no savepoint directory configured.", jobGraph.getJobID()); - - return FutureUtils.completedExceptionally(new IllegalStateException( - "No savepoint directory configured. You can either specify a directory " + - "while cancelling via -s :targetDirectory or configure a cluster-wide " + - "default via key '" + CheckpointingOptions.SAVEPOINT_DIRECTORY.key() + "'.")); - } - - // we stop the checkpoint coordinator so that we are guaranteed - // to have only the data of the synchronous savepoint committed. - // in case of failure, and if the job restarts, the coordinator - // will be restarted by the CheckpointCoordinatorDeActivator. - checkpointCoordinator.stopCheckpointScheduler(); - - final long now = System.currentTimeMillis(); - final CompletableFuture savepointFuture = checkpointCoordinator - .triggerSynchronousSavepoint(now, advanceToEndOfEventTime, targetDirectory) - .thenApply(CompletedCheckpoint::getExternalPointer); - - final CompletableFuture terminationFuture = executionGraph - .getTerminationFuture() - .handle((jobstatus, throwable) -> { - - if (throwable != null) { - log.info("Failed during stopping job {} with a savepoint. Reason: {}", jobGraph.getJobID(), throwable.getMessage()); - throw new CompletionException(throwable); - } else if (jobstatus != JobStatus.FINISHED) { - log.info("Failed during stopping job {} with a savepoint. Reason: Reached state {} instead of FINISHED.", jobGraph.getJobID(), jobstatus); - throw new CompletionException(new FlinkException("Reached state " + jobstatus + " instead of FINISHED.")); - } - return jobstatus; - }); - - return savepointFuture.thenCompose((path) -> - terminationFuture.thenApply((jobStatus -> path))); - } - - private String retrieveTaskManagerLocation(ExecutionAttemptID executionAttemptID) { - final Optional currentExecution = Optional.ofNullable(executionGraph.getRegisteredExecutions().get(executionAttemptID)); - - return currentExecution - .map(Execution::getAssignedResourceLocation) - .map(TaskManagerLocation::toString) - .orElse("Unknown location"); - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java new file mode 100644 index 0000000000..6c2ed517e1 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java @@ -0,0 +1,644 @@ +/* + * 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.scheduler; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.queryablestate.KvStateID; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; +import org.apache.flink.runtime.blob.BlobWriter; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; +import org.apache.flink.runtime.checkpoint.CheckpointMetrics; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder; +import org.apache.flink.runtime.executiongraph.ExecutionGraphException; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.IntermediateResult; +import org.apache.flink.runtime.executiongraph.JobStatusListener; +import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; +import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; +import org.apache.flink.runtime.executiongraph.restart.RestartStrategyResolving; +import org.apache.flink.runtime.io.network.partition.PartitionTracker; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.jobmanager.PartitionProducerDisposedException; +import org.apache.flink.runtime.jobmaster.SerializedInputSplit; +import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; +import org.apache.flink.runtime.messages.FlinkJobNotFoundException; +import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; +import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; +import org.apache.flink.runtime.messages.webmonitor.JobDetails; +import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; +import org.apache.flink.runtime.query.KvStateLocation; +import org.apache.flink.runtime.query.KvStateLocationRegistry; +import org.apache.flink.runtime.query.UnknownKvStateLocation; +import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker; +import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStats; +import org.apache.flink.runtime.shuffle.ShuffleMaster; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.webmonitor.WebMonitorUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.function.FunctionUtils; + +import org.slf4j.Logger; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Base class which can be used to implement {@link SchedulerNG}. + */ +public abstract class SchedulerBase implements SchedulerNG { + + private final Logger log; + + private final JobGraph jobGraph; + + private final ExecutionGraph executionGraph; + + private final BackPressureStatsTracker backPressureStatsTracker; + + private final Executor ioExecutor; + + private final Configuration jobMasterConfiguration; + + private final SlotProvider slotProvider; + + private final ScheduledExecutorService futureExecutor; + + private final ClassLoader userCodeLoader; + + private final CheckpointRecoveryFactory checkpointRecoveryFactory; + + private final Time rpcTimeout; + + private final RestartStrategy restartStrategy; + + private final BlobWriter blobWriter; + + private final Time slotRequestTimeout; + + private ComponentMainThreadExecutor mainThreadExecutor = new ComponentMainThreadExecutor.DummyComponentMainThreadExecutor( + "LegacyScheduler is not initialized with proper main thread executor. " + + "Call to LegacyScheduler.setMainThreadExecutor(...) required."); + + public SchedulerBase( + final Logger log, + final JobGraph jobGraph, + final BackPressureStatsTracker backPressureStatsTracker, + final Executor ioExecutor, + final Configuration jobMasterConfiguration, + final SlotProvider slotProvider, + final ScheduledExecutorService futureExecutor, + final ClassLoader userCodeLoader, + final CheckpointRecoveryFactory checkpointRecoveryFactory, + final Time rpcTimeout, + final RestartStrategyFactory restartStrategyFactory, + final BlobWriter blobWriter, + final JobManagerJobMetricGroup jobManagerJobMetricGroup, + final Time slotRequestTimeout, + final ShuffleMaster shuffleMaster, + final PartitionTracker partitionTracker) throws Exception { + + this.log = checkNotNull(log); + this.jobGraph = checkNotNull(jobGraph); + this.backPressureStatsTracker = checkNotNull(backPressureStatsTracker); + this.ioExecutor = checkNotNull(ioExecutor); + this.jobMasterConfiguration = checkNotNull(jobMasterConfiguration); + this.slotProvider = checkNotNull(slotProvider); + this.futureExecutor = checkNotNull(futureExecutor); + this.userCodeLoader = checkNotNull(userCodeLoader); + this.checkpointRecoveryFactory = checkNotNull(checkpointRecoveryFactory); + this.rpcTimeout = checkNotNull(rpcTimeout); + + final RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration = + jobGraph.getSerializedExecutionConfig() + .deserializeValue(userCodeLoader) + .getRestartStrategy(); + + this.restartStrategy = RestartStrategyResolving.resolve(restartStrategyConfiguration, + restartStrategyFactory, + jobGraph.isCheckpointingEnabled()); + + log.info("Using restart strategy {} for {} ({}).", this.restartStrategy, jobGraph.getName(), jobGraph.getJobID()); + + this.blobWriter = checkNotNull(blobWriter); + this.slotRequestTimeout = checkNotNull(slotRequestTimeout); + + this.executionGraph = createAndRestoreExecutionGraph(jobManagerJobMetricGroup, checkNotNull(shuffleMaster), checkNotNull(partitionTracker)); + } + + private ExecutionGraph createAndRestoreExecutionGraph( + JobManagerJobMetricGroup currentJobManagerJobMetricGroup, + ShuffleMaster shuffleMaster, + PartitionTracker partitionTracker) throws Exception { + + ExecutionGraph newExecutionGraph = createExecutionGraph(currentJobManagerJobMetricGroup, shuffleMaster, partitionTracker); + + final CheckpointCoordinator checkpointCoordinator = newExecutionGraph.getCheckpointCoordinator(); + + if (checkpointCoordinator != null) { + // check whether we find a valid checkpoint + if (!checkpointCoordinator.restoreLatestCheckpointedState( + newExecutionGraph.getAllVertices(), + false, + false)) { + + // check whether we can restore from a savepoint + tryRestoreExecutionGraphFromSavepoint(newExecutionGraph, jobGraph.getSavepointRestoreSettings()); + } + } + + return newExecutionGraph; + } + + private ExecutionGraph createExecutionGraph( + JobManagerJobMetricGroup currentJobManagerJobMetricGroup, + ShuffleMaster shuffleMaster, + final PartitionTracker partitionTracker) throws JobExecutionException, JobException { + return ExecutionGraphBuilder.buildGraph( + null, + jobGraph, + jobMasterConfiguration, + futureExecutor, + ioExecutor, + slotProvider, + userCodeLoader, + checkpointRecoveryFactory, + rpcTimeout, + restartStrategy, + currentJobManagerJobMetricGroup, + blobWriter, + slotRequestTimeout, + log, + shuffleMaster, + partitionTracker); + } + + /** + * Tries to restore the given {@link ExecutionGraph} from the provided {@link SavepointRestoreSettings}. + * + * @param executionGraphToRestore {@link ExecutionGraph} which is supposed to be restored + * @param savepointRestoreSettings {@link SavepointRestoreSettings} containing information about the savepoint to restore from + * @throws Exception if the {@link ExecutionGraph} could not be restored + */ + private void tryRestoreExecutionGraphFromSavepoint(ExecutionGraph executionGraphToRestore, SavepointRestoreSettings savepointRestoreSettings) throws Exception { + if (savepointRestoreSettings.restoreSavepoint()) { + final CheckpointCoordinator checkpointCoordinator = executionGraphToRestore.getCheckpointCoordinator(); + if (checkpointCoordinator != null) { + checkpointCoordinator.restoreSavepoint( + savepointRestoreSettings.getRestorePath(), + savepointRestoreSettings.allowNonRestoredState(), + executionGraphToRestore.getAllVertices(), + userCodeLoader); + } + } + } + + @Override + public void setMainThreadExecutor(final ComponentMainThreadExecutor mainThreadExecutor) { + this.mainThreadExecutor = checkNotNull(mainThreadExecutor); + executionGraph.start(mainThreadExecutor); + } + + @Override + public void registerJobStatusListener(final JobStatusListener jobStatusListener) { + executionGraph.registerJobStatusListener(jobStatusListener); + } + + @Override + public void startScheduling() { + mainThreadExecutor.assertRunningInMainThread(); + + try { + executionGraph.scheduleForExecution(); + } + catch (Throwable t) { + executionGraph.failGlobal(t); + } + } + + @Override + public void suspend(Throwable cause) { + mainThreadExecutor.assertRunningInMainThread(); + executionGraph.suspend(cause); + } + + @Override + public void cancel() { + mainThreadExecutor.assertRunningInMainThread(); + executionGraph.cancel(); + } + + @Override + public CompletableFuture getTerminationFuture() { + return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn()); + } + + @Override + public boolean updateTaskExecutionState(final TaskExecutionState taskExecutionState) { + mainThreadExecutor.assertRunningInMainThread(); + return executionGraph.updateState(taskExecutionState); + } + + @Override + public SerializedInputSplit requestNextInputSplit(JobVertexID vertexID, ExecutionAttemptID executionAttempt) throws IOException { + mainThreadExecutor.assertRunningInMainThread(); + + final Execution execution = executionGraph.getRegisteredExecutions().get(executionAttempt); + if (execution == null) { + // can happen when JobManager had already unregistered this execution upon on task failure, + // but TaskManager get some delay to aware of that situation + if (log.isDebugEnabled()) { + log.debug("Can not find Execution for attempt {}.", executionAttempt); + } + // but we should TaskManager be aware of this + throw new IllegalArgumentException("Can not find Execution for attempt " + executionAttempt); + } + + final ExecutionJobVertex vertex = executionGraph.getJobVertex(vertexID); + if (vertex == null) { + throw new IllegalArgumentException("Cannot find execution vertex for vertex ID " + vertexID); + } + + if (vertex.getSplitAssigner() == null) { + throw new IllegalStateException("No InputSplitAssigner for vertex ID " + vertexID); + } + + final InputSplit nextInputSplit = execution.getNextInputSplit(); + + if (log.isDebugEnabled()) { + log.debug("Send next input split {}.", nextInputSplit); + } + + try { + final byte[] serializedInputSplit = InstantiationUtil.serializeObject(nextInputSplit); + return new SerializedInputSplit(serializedInputSplit); + } catch (Exception ex) { + IOException reason = new IOException("Could not serialize the next input split of class " + + nextInputSplit.getClass() + ".", ex); + vertex.fail(reason); + throw reason; + } + } + + @Override + public ExecutionState requestPartitionState( + final IntermediateDataSetID intermediateResultId, + final ResultPartitionID resultPartitionId) throws PartitionProducerDisposedException { + + mainThreadExecutor.assertRunningInMainThread(); + + final Execution execution = executionGraph.getRegisteredExecutions().get(resultPartitionId.getProducerId()); + if (execution != null) { + return execution.getState(); + } + else { + final IntermediateResult intermediateResult = + executionGraph.getAllIntermediateResults().get(intermediateResultId); + + if (intermediateResult != null) { + // Try to find the producing execution + Execution producerExecution = intermediateResult + .getPartitionById(resultPartitionId.getPartitionId()) + .getProducer() + .getCurrentExecutionAttempt(); + + if (producerExecution.getAttemptId().equals(resultPartitionId.getProducerId())) { + return producerExecution.getState(); + } else { + throw new PartitionProducerDisposedException(resultPartitionId); + } + } else { + throw new IllegalArgumentException("Intermediate data set with ID " + + intermediateResultId + " not found."); + } + } + } + + @Override + public void scheduleOrUpdateConsumers(final ResultPartitionID partitionID) { + mainThreadExecutor.assertRunningInMainThread(); + + try { + executionGraph.scheduleOrUpdateConsumers(partitionID); + } catch (ExecutionGraphException e) { + throw new RuntimeException(e); + } + } + + @Override + public ArchivedExecutionGraph requestJob() { + mainThreadExecutor.assertRunningInMainThread(); + return ArchivedExecutionGraph.createFrom(executionGraph); + } + + @Override + public JobStatus requestJobStatus() { + return executionGraph.getState(); + } + + @Override + public JobDetails requestJobDetails() { + mainThreadExecutor.assertRunningInMainThread(); + return WebMonitorUtils.createDetailsForJob(executionGraph); + } + + @Override + public KvStateLocation requestKvStateLocation(final JobID jobId, final String registrationName) throws UnknownKvStateLocation, FlinkJobNotFoundException { + mainThreadExecutor.assertRunningInMainThread(); + + // sanity check for the correct JobID + if (jobGraph.getJobID().equals(jobId)) { + if (log.isDebugEnabled()) { + log.debug("Lookup key-value state for job {} with registration " + + "name {}.", jobGraph.getJobID(), registrationName); + } + + final KvStateLocationRegistry registry = executionGraph.getKvStateLocationRegistry(); + final KvStateLocation location = registry.getKvStateLocation(registrationName); + if (location != null) { + return location; + } else { + throw new UnknownKvStateLocation(registrationName); + } + } else { + if (log.isDebugEnabled()) { + log.debug("Request of key-value state location for unknown job {} received.", jobId); + } + throw new FlinkJobNotFoundException(jobId); + } + } + + @Override + public void notifyKvStateRegistered(final JobID jobId, final JobVertexID jobVertexId, final KeyGroupRange keyGroupRange, final String registrationName, final KvStateID kvStateId, final InetSocketAddress kvStateServerAddress) throws FlinkJobNotFoundException { + mainThreadExecutor.assertRunningInMainThread(); + + if (jobGraph.getJobID().equals(jobId)) { + if (log.isDebugEnabled()) { + log.debug("Key value state registered for job {} under name {}.", + jobGraph.getJobID(), registrationName); + } + + try { + executionGraph.getKvStateLocationRegistry().notifyKvStateRegistered( + jobVertexId, keyGroupRange, registrationName, kvStateId, kvStateServerAddress); + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + throw new FlinkJobNotFoundException(jobId); + } + } + + @Override + public void notifyKvStateUnregistered(final JobID jobId, final JobVertexID jobVertexId, final KeyGroupRange keyGroupRange, final String registrationName) throws FlinkJobNotFoundException { + mainThreadExecutor.assertRunningInMainThread(); + + if (jobGraph.getJobID().equals(jobId)) { + if (log.isDebugEnabled()) { + log.debug("Key value state unregistered for job {} under name {}.", + jobGraph.getJobID(), registrationName); + } + + try { + executionGraph.getKvStateLocationRegistry().notifyKvStateUnregistered( + jobVertexId, keyGroupRange, registrationName); + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + throw new FlinkJobNotFoundException(jobId); + } + } + + @Override + public void updateAccumulators(final AccumulatorSnapshot accumulatorSnapshot) { + mainThreadExecutor.assertRunningInMainThread(); + + executionGraph.updateAccumulators(accumulatorSnapshot); + } + + @Override + public Optional requestOperatorBackPressureStats(final JobVertexID jobVertexId) throws FlinkException { + final ExecutionJobVertex jobVertex = executionGraph.getJobVertex(jobVertexId); + if (jobVertex == null) { + throw new FlinkException("JobVertexID not found " + + jobVertexId); + } + + return backPressureStatsTracker.getOperatorBackPressureStats(jobVertex); + } + + @Override + public CompletableFuture triggerSavepoint(final String targetDirectory, final boolean cancelJob) { + mainThreadExecutor.assertRunningInMainThread(); + + final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); + if (checkpointCoordinator == null) { + throw new IllegalStateException( + String.format("Job %s is not a streaming job.", jobGraph.getJobID())); + } else if (targetDirectory == null && !checkpointCoordinator.getCheckpointStorage().hasDefaultSavepointLocation()) { + log.info("Trying to cancel job {} with savepoint, but no savepoint directory configured.", jobGraph.getJobID()); + + throw new IllegalStateException( + "No savepoint directory configured. You can either specify a directory " + + "while cancelling via -s :targetDirectory or configure a cluster-wide " + + "default via key '" + CheckpointingOptions.SAVEPOINT_DIRECTORY.key() + "'."); + } + + if (cancelJob) { + checkpointCoordinator.stopCheckpointScheduler(); + } + + return checkpointCoordinator + .triggerSavepoint(System.currentTimeMillis(), targetDirectory) + .thenApply(CompletedCheckpoint::getExternalPointer) + .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(); + } + return path; + }, mainThreadExecutor); + } + + private void startCheckpointScheduler(final CheckpointCoordinator checkpointCoordinator) { + mainThreadExecutor.assertRunningInMainThread(); + + if (checkpointCoordinator.isPeriodicCheckpointingConfigured()) { + try { + checkpointCoordinator.startCheckpointScheduler(); + } catch (IllegalStateException ignored) { + // Concurrent shut down of the coordinator + } + } + } + + @Override + public void acknowledgeCheckpoint(final JobID jobID, final ExecutionAttemptID executionAttemptID, final long checkpointId, final CheckpointMetrics checkpointMetrics, final TaskStateSnapshot checkpointState) { + mainThreadExecutor.assertRunningInMainThread(); + + final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); + final AcknowledgeCheckpoint ackMessage = new AcknowledgeCheckpoint( + jobID, + executionAttemptID, + checkpointId, + checkpointMetrics, + checkpointState); + + final String taskManagerLocationInfo = retrieveTaskManagerLocation(executionAttemptID); + + if (checkpointCoordinator != null) { + ioExecutor.execute(() -> { + try { + checkpointCoordinator.receiveAcknowledgeMessage(ackMessage, taskManagerLocationInfo); + } catch (Throwable t) { + log.warn("Error while processing checkpoint acknowledgement message", t); + } + }); + } else { + 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()); + } + } + } + + @Override + public void declineCheckpoint(final DeclineCheckpoint decline) { + mainThreadExecutor.assertRunningInMainThread(); + + final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); + final String taskManagerLocationInfo = retrieveTaskManagerLocation(decline.getTaskExecutionId()); + + if (checkpointCoordinator != null) { + ioExecutor.execute(() -> { + try { + checkpointCoordinator.receiveDeclineMessage(decline, taskManagerLocationInfo); + } catch (Exception e) { + log.error("Error in CheckpointCoordinator while processing {}", decline, e); + } + }); + } else { + 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()); + } + } + } + + @Override + public CompletableFuture stopWithSavepoint(final String targetDirectory, final boolean advanceToEndOfEventTime) { + mainThreadExecutor.assertRunningInMainThread(); + + final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); + + if (checkpointCoordinator == null) { + return FutureUtils.completedExceptionally(new IllegalStateException( + String.format("Job %s is not a streaming job.", jobGraph.getJobID()))); + } + + if (targetDirectory == null && !checkpointCoordinator.getCheckpointStorage().hasDefaultSavepointLocation()) { + log.info("Trying to cancel job {} with savepoint, but no savepoint directory configured.", jobGraph.getJobID()); + + return FutureUtils.completedExceptionally(new IllegalStateException( + "No savepoint directory configured. You can either specify a directory " + + "while cancelling via -s :targetDirectory or configure a cluster-wide " + + "default via key '" + CheckpointingOptions.SAVEPOINT_DIRECTORY.key() + "'.")); + } + + // we stop the checkpoint coordinator so that we are guaranteed + // to have only the data of the synchronous savepoint committed. + // in case of failure, and if the job restarts, the coordinator + // will be restarted by the CheckpointCoordinatorDeActivator. + checkpointCoordinator.stopCheckpointScheduler(); + + final long now = System.currentTimeMillis(); + final CompletableFuture savepointFuture = checkpointCoordinator + .triggerSynchronousSavepoint(now, advanceToEndOfEventTime, targetDirectory) + .thenApply(CompletedCheckpoint::getExternalPointer); + + final CompletableFuture terminationFuture = executionGraph + .getTerminationFuture() + .handle((jobstatus, throwable) -> { + + if (throwable != null) { + log.info("Failed during stopping job {} with a savepoint. Reason: {}", jobGraph.getJobID(), throwable.getMessage()); + throw new CompletionException(throwable); + } else if (jobstatus != JobStatus.FINISHED) { + log.info("Failed during stopping job {} with a savepoint. Reason: Reached state {} instead of FINISHED.", jobGraph.getJobID(), jobstatus); + throw new CompletionException(new FlinkException("Reached state " + jobstatus + " instead of FINISHED.")); + } + return jobstatus; + }); + + return savepointFuture.thenCompose((path) -> + terminationFuture.thenApply((jobStatus -> path))); + } + + private String retrieveTaskManagerLocation(ExecutionAttemptID executionAttemptID) { + final Optional currentExecution = Optional.ofNullable(executionGraph.getRegisteredExecutions().get(executionAttemptID)); + + return currentExecution + .map(Execution::getAssignedResourceLocation) + .map(TaskManagerLocation::toString) + .orElse("Unknown location"); + } + +} -- Gitee From c546448a360ec19d0e9c9ba3fa7d84068367bbe5 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Thu, 5 Sep 2019 09:46:39 +0200 Subject: [PATCH 059/268] [FLINK-12433][runtime] Add additional implementation to DefaultScheduler Implement DefaultScheduler to the point where streaming and batch WordCount can be executed. Note that restoring state does not work yet. This will be fixed in later commits. Add additional functionality to ExecutionGraph so that it can be used by DefaultScheduler. Some of the logic in ExecutionGraph must not be run when DefaultScheduler is configured. We introduce a function isLegacyScheduling() to be able to toggle off some of the legacy behavior when DefaultScheduler is configured. This closes #9663. --- .../runtime/executiongraph/Execution.java | 73 +++- .../executiongraph/ExecutionGraph.java | 70 +++- .../executiongraph/ExecutionVertex.java | 127 +++--- .../flip1/RestartPipelinedRegionStrategy.java | 8 + .../DefaultExecutionVertexOperations.java | 38 ++ .../runtime/scheduler/DefaultScheduler.java | 363 +++++++++++++++++- .../scheduler/DefaultSchedulerFactory.java | 29 +- .../runtime/scheduler/DeploymentHandle.java | 83 ++++ .../scheduler/ExecutionVertexOperations.java | 35 ++ ...ionVertexSchedulingRequirementsMapper.java | 58 +++ .../InternalTaskFailuresListener.java | 37 ++ .../runtime/scheduler/LegacyScheduler.java | 65 +--- .../runtime/scheduler/SchedulerBase.java | 139 ++++++- ...dulerNgOnInternalTaskFailuresListener.java | 54 +++ .../ExecutionVertexDeploymentTest.java | 2 +- .../flip1/TestRestartBackoffTimeStrategy.java | 12 +- .../scheduler/DefaultSchedulerTest.java | 316 +++++++++++++++ ...ingExecutionVertexOperationsDecorator.java | 78 ++++ .../SubmissionTrackingTaskManagerGateway.java | 94 +++++ 19 files changed, 1524 insertions(+), 157 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionVertexOperations.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DeploymentHandle.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionVertexOperations.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionVertexSchedulingRequirementsMapper.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/InternalTaskFailuresListener.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/UpdateSchedulerNgOnInternalTaskFailuresListener.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/FailingExecutionVertexOperationsDecorator.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SubmissionTrackingTaskManagerGateway.java 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 7ba2353d32..503b59e288 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 @@ -63,6 +63,7 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.OptionalFailure; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.ThrowingRunnable; import org.slf4j.Logger; @@ -295,8 +296,7 @@ public class Execution implements AccessExecution, Archiveable registerProducedPartitions(TaskManagerLocation location) { + public CompletableFuture registerProducedPartitions(TaskManagerLocation location) { + Preconditions.checkState(isLegacyScheduling()); + return registerProducedPartitions(location, vertex.getExecutionGraph().getScheduleMode().allowLazyDeployment()); + } + + public CompletableFuture registerProducedPartitions( + TaskManagerLocation location, + boolean sendScheduleOrUpdateConsumersMessage) { + assertRunningInJobMasterMainThread(); return FutureUtils.thenApplyAsyncIfNotDone( - registerProducedPartitions(vertex, location, attemptId), + registerProducedPartitions(vertex, location, attemptId, sendScheduleOrUpdateConsumersMessage), vertex.getExecutionGraph().getJobMasterMainThreadExecutor(), producedPartitionsCache -> { producedPartitions = producedPartitionsCache; @@ -615,10 +622,10 @@ public class Execution implements AccessExecution, Archiveable> registerProducedPartitions( ExecutionVertex vertex, TaskManagerLocation location, - ExecutionAttemptID attemptId) { - ProducerDescriptor producerDescriptor = ProducerDescriptor.create(location, attemptId); + ExecutionAttemptID attemptId, + boolean sendScheduleOrUpdateConsumersMessage) { - boolean lazyScheduling = vertex.getExecutionGraph().getScheduleMode().allowLazyDeployment(); + ProducerDescriptor producerDescriptor = ProducerDescriptor.create(location, attemptId); Collection partitions = vertex.getProducedPartitions().values(); Collection> partitionRegistrations = @@ -637,7 +644,7 @@ public class Execution implements AccessExecution, Archiveable> userAccumulators, IOMetrics metrics) { - // skip release of partitions since this is only called if the TM actually sent the FAILED state update - // in this case all partitions have already been cleaned up - processFail(t, true, userAccumulators, metrics, false); + markFailed(t, userAccumulators, metrics, false); + } + + void markFailed(Throwable t, Map> userAccumulators, IOMetrics metrics, boolean fromSchedulerNg) { + processFail(t, true, userAccumulators, metrics, false, fromSchedulerNg); } @VisibleForTesting @@ -1179,11 +1200,15 @@ public class Execution implements AccessExecution, Archiveable> userAccumulators, IOMetrics metrics, boolean releasePartitions) { + private void processFail(Throwable t, boolean isCallback, Map> userAccumulators, IOMetrics metrics, boolean releasePartitions, boolean fromSchedulerNg) { // damn, we failed. This means only that we keep our books and notify our parent JobExecutionVertex // the actual computation on the task manager is cleaned up by the TaskManager that noticed the failure @@ -1211,7 +1236,15 @@ public class Execution implements AccessExecution, Archiveable { + transitionState(JobStatus.FAILED, cause); + onTerminalState(JobStatus.FAILED); + })); + } + private void onTerminalState(JobStatus status) { try { CheckpointCoordinator coord = this.checkpointCoordinator; @@ -1557,7 +1611,7 @@ public class ExecutionGraph implements AccessExecutionGraph { case FAILED: // this deserialization is exception-free accumulators = deserializeAccumulators(state); - attempt.markFailed(state.getError(userClassLoader), accumulators, state.getIOMetrics()); + attempt.markFailed(state.getError(userClassLoader), accumulators, state.getIOMetrics(), !isLegacyScheduling()); return true; default: @@ -1727,6 +1781,10 @@ public class ExecutionGraph implements AccessExecutionGraph { final ExecutionState newExecutionState, final Throwable error) { + if (!isLegacyScheduling()) { + return; + } + // see what this means for us. currently, the first FAILED state means -> FAILED if (newExecutionState == ExecutionState.FAILED) { final Throwable ex = error != null ? error : new FlinkException("Unknown Error (missing cause)"); @@ -1757,6 +1815,12 @@ public class ExecutionGraph implements AccessExecutionGraph { } } + void notifySchedulerNgAboutInternalTaskFailure(final ExecutionAttemptID attemptId, final Throwable t) { + if (internalTaskFailuresListener != null) { + internalTaskFailuresListener.notifyFailed(attemptId, t); + } + } + ShuffleMaster getShuffleMaster() { return shuffleMaster; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index 6d262ee58d..500691b049 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -58,6 +58,7 @@ import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.stream.IntStream; @@ -514,6 +515,18 @@ public class ExecutionVertex implements AccessExecutionVertex, Archiveable getPreferredLocationBasedOnState() { + if (currentExecution.getTaskRestore() != null) { + return Optional.ofNullable(getLatestPriorLocation()); + } + + return Optional.empty(); + } + /** * Gets the location preferences of the vertex's current task execution, as determined by the locations * of the predecessors from which it receives input data. @@ -599,61 +612,75 @@ public class ExecutionVertex implements AccessExecutionVertex, Archiveable cancel(final ExecutionVertex executionVertex) { + return executionVertex.cancel(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java index 759a56a2f4..0d0af10609 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java @@ -23,40 +23,92 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobWriter; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.failover.flip1.ExecutionFailureHandler; +import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverStrategy; +import org.apache.flink.runtime.executiongraph.failover.flip1.FailureHandlingResult; +import org.apache.flink.runtime.executiongraph.failover.flip1.RestartBackoffTimeStrategy; import org.apache.flink.runtime.executiongraph.restart.ThrowingRestartStrategy; import org.apache.flink.runtime.io.network.partition.PartitionTracker; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.LazyFromSourcesSchedulingStrategy; +import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategy; +import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategyFactory; import org.apache.flink.runtime.shuffle.ShuffleMaster; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.slf4j.Logger; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; /** - * Stub implementation of the future default scheduler. + * The future default scheduler. */ -public class DefaultScheduler extends SchedulerBase { +public class DefaultScheduler extends SchedulerBase implements SchedulerOperations { + + private final Logger log; + + private final ClassLoader userCodeLoader; + + private final ExecutionSlotAllocator executionSlotAllocator; + + private final ExecutionFailureHandler executionFailureHandler; + + private final ScheduledExecutor delayExecutor; + + private final SchedulingStrategy schedulingStrategy; + + private final ExecutionVertexVersioner executionVertexVersioner; + + private final ExecutionVertexOperations executionVertexOperations; public DefaultScheduler( - final Logger log, - final JobGraph jobGraph, - final BackPressureStatsTracker backPressureStatsTracker, - final Executor ioExecutor, - final Configuration jobMasterConfiguration, - final SlotProvider slotProvider, - final ScheduledExecutorService futureExecutor, - final ClassLoader userCodeLoader, - final CheckpointRecoveryFactory checkpointRecoveryFactory, - final Time rpcTimeout, - final BlobWriter blobWriter, - final JobManagerJobMetricGroup jobManagerJobMetricGroup, - final Time slotRequestTimeout, - final ShuffleMaster shuffleMaster, - final PartitionTracker partitionTracker) throws Exception { + final Logger log, + final JobGraph jobGraph, + final BackPressureStatsTracker backPressureStatsTracker, + final Executor ioExecutor, + final Configuration jobMasterConfiguration, + final SlotProvider slotProvider, + final ScheduledExecutorService futureExecutor, + final ScheduledExecutor delayExecutor, + final ClassLoader userCodeLoader, + final CheckpointRecoveryFactory checkpointRecoveryFactory, + final Time rpcTimeout, + final BlobWriter blobWriter, + final JobManagerJobMetricGroup jobManagerJobMetricGroup, + final Time slotRequestTimeout, + final ShuffleMaster shuffleMaster, + final PartitionTracker partitionTracker, + final SchedulingStrategyFactory schedulingStrategyFactory, + final FailoverStrategy.Factory failoverStrategyFactory, + final RestartBackoffTimeStrategy restartBackoffTimeStrategy, + final ExecutionVertexOperations executionVertexOperations, + final ExecutionVertexVersioner executionVertexVersioner) throws Exception { super( log, @@ -75,10 +127,283 @@ public class DefaultScheduler extends SchedulerBase { slotRequestTimeout, shuffleMaster, partitionTracker); + + this.log = log; + + this.delayExecutor = checkNotNull(delayExecutor); + this.userCodeLoader = checkNotNull(userCodeLoader); + this.executionVertexOperations = checkNotNull(executionVertexOperations); + this.executionVertexVersioner = checkNotNull(executionVertexVersioner); + + this.executionFailureHandler = new ExecutionFailureHandler(failoverStrategyFactory.create(getFailoverTopology()), restartBackoffTimeStrategy); + this.schedulingStrategy = schedulingStrategyFactory.createInstance(this, getSchedulingTopology(), getJobGraph()); + this.executionSlotAllocator = new DefaultExecutionSlotAllocator(slotProvider, getInputsLocationsRetriever(), slotRequestTimeout); + } + + // ------------------------------------------------------------------------ + // SchedulerNG + // ------------------------------------------------------------------------ + + @Override + protected void startSchedulingInternal() { + log.debug("Starting scheduling with scheduling strategy [{}]", schedulingStrategy.getClass().getName()); + prepareExecutionGraphForNgScheduling(); + schedulingStrategy.startScheduling(); + } + + @Override + protected void updateTaskExecutionStateInternal(final ExecutionVertexID executionVertexId, final TaskExecutionState taskExecutionState) { + schedulingStrategy.onExecutionStateChange(executionVertexId, taskExecutionState.getExecutionState()); + maybeHandleTaskFailure(taskExecutionState, executionVertexId); + } + + private void maybeHandleTaskFailure(final TaskExecutionState taskExecutionState, final ExecutionVertexID executionVertexId) { + if (taskExecutionState.getExecutionState() == ExecutionState.FAILED) { + final Throwable error = taskExecutionState.getError(userCodeLoader); + handleTaskFailure(executionVertexId, error); + } } + private void handleTaskFailure(final ExecutionVertexID executionVertexId, final Throwable error) { + final FailureHandlingResult failureHandlingResult = executionFailureHandler.getFailureHandlingResult(executionVertexId, error); + maybeRestartTasks(failureHandlingResult); + } + + private void maybeRestartTasks(final FailureHandlingResult failureHandlingResult) { + if (failureHandlingResult.canRestart()) { + restartTasksWithDelay(failureHandlingResult); + } else { + failJob(failureHandlingResult.getError()); + } + } + + private void restartTasksWithDelay(final FailureHandlingResult failureHandlingResult) { + final Set verticesToRestart = failureHandlingResult.getVerticesToRestart(); + + final Set executionVertexVersions = + new HashSet<>(executionVertexVersioner.recordVertexModifications(verticesToRestart).values()); + + final CompletableFuture cancelFuture = cancelTasksAsync(verticesToRestart); + + delayExecutor.schedule( + () -> FutureUtils.assertNoException( + cancelFuture.thenRunAsync(restartTasks(executionVertexVersions), getMainThreadExecutor())), + failureHandlingResult.getRestartDelayMS(), + TimeUnit.MILLISECONDS); + } + + private Runnable restartTasks(final Set executionVertexVersions) { + return () -> { + final Set verticesToRestart = executionVertexVersioner.getUnmodifiedExecutionVertices(executionVertexVersions); + schedulingStrategy.restartTasks(verticesToRestart); + }; + } + + private CompletableFuture cancelTasksAsync(final Set verticesToRestart) { + final List> cancelFutures = verticesToRestart.stream() + .map(this::cancelExecutionVertex) + .collect(Collectors.toList()); + + return FutureUtils.combineAll(cancelFutures); + } + + private CompletableFuture cancelExecutionVertex(final ExecutionVertexID executionVertexId) { + return executionVertexOperations.cancel(getExecutionVertex(executionVertexId)); + } + + @Override + protected void scheduleOrUpdateConsumersInternal(final ExecutionVertexID producerVertexId, final ResultPartitionID partitionId) { + schedulingStrategy.onPartitionConsumable(producerVertexId, partitionId); + } + + // ------------------------------------------------------------------------ + // SchedulerOperations + // ------------------------------------------------------------------------ + @Override - public void startScheduling() { - throw new UnsupportedOperationException(); + public void allocateSlotsAndDeploy(final Collection executionVertexDeploymentOptions) { + final Map deploymentOptionsByVertex = groupDeploymentOptionsByVertexId(executionVertexDeploymentOptions); + final Set verticesToDeploy = deploymentOptionsByVertex.keySet(); + final Map requiredVersionByVertex = executionVertexVersioner.recordVertexModifications(verticesToDeploy); + + prepareToDeployVertices(verticesToDeploy); + + final Collection slotExecutionVertexAssignments = allocateSlots(executionVertexDeploymentOptions); + + final Collection deploymentHandles = createDeploymentHandles( + requiredVersionByVertex, + deploymentOptionsByVertex, + slotExecutionVertexAssignments); + + if (isDeployIndividually()) { + deployIndividually(deploymentHandles); + } else { + waitForAllSlotsAndDeploy(deploymentHandles); + } + } + + private static Map groupDeploymentOptionsByVertexId( + final Collection executionVertexDeploymentOptions) { + return executionVertexDeploymentOptions.stream().collect(Collectors.toMap( + ExecutionVertexDeploymentOption::getExecutionVertexId, + Function.identity())); + } + + private void prepareToDeployVertices(final Set verticesToDeploy) { + cancelSlotAssignments(verticesToDeploy); + resetForNewExecutionIfInTerminalState(verticesToDeploy); + transitionToScheduled(verticesToDeploy); + } + + private void cancelSlotAssignments(final Collection vertices) { + vertices.forEach(executionSlotAllocator::cancel); + } + + private Collection allocateSlots(final Collection executionVertexDeploymentOptions) { + return executionSlotAllocator.allocateSlotsFor(executionVertexDeploymentOptions + .stream() + .map(ExecutionVertexDeploymentOption::getExecutionVertexId) + .map(this::getExecutionVertex) + .map(ExecutionVertexSchedulingRequirementsMapper::from) + .collect(Collectors.toList())); + } + + private static Collection createDeploymentHandles( + final Map requiredVersionByVertex, + final Map deploymentOptionsByVertex, + final Collection slotExecutionVertexAssignments) { + + return slotExecutionVertexAssignments + .stream() + .map(slotExecutionVertexAssignment -> { + final ExecutionVertexID executionVertexId = slotExecutionVertexAssignment.getExecutionVertexId(); + return new DeploymentHandle( + requiredVersionByVertex.get(executionVertexId), + deploymentOptionsByVertex.get(executionVertexId), + slotExecutionVertexAssignment); + }) + .collect(Collectors.toList()); + } + + /** + * HACK: See FLINK-14162 + * for details. + */ + private boolean isDeployIndividually() { + return schedulingStrategy instanceof LazyFromSourcesSchedulingStrategy; + } + + private void deployIndividually(final Collection deploymentHandles) { + for (final DeploymentHandle deploymentHandle : deploymentHandles) { + FutureUtils.assertNoException( + deploymentHandle + .getSlotExecutionVertexAssignment() + .getLogicalSlotFuture() + .handle(assignResourceOrHandleError(deploymentHandle)) + .handle(deployOrHandleError(deploymentHandle))); + } + } + + private void waitForAllSlotsAndDeploy(final Collection deploymentHandles) { + FutureUtils.assertNoException( + assignAllResources(deploymentHandles).handle(deployAll(deploymentHandles))); + } + + private CompletableFuture assignAllResources(final Collection deploymentHandles) { + final List> slotAssignedFutures = new ArrayList<>(); + for (DeploymentHandle deploymentHandle : deploymentHandles) { + final CompletableFuture slotAssigned = deploymentHandle + .getSlotExecutionVertexAssignment() + .getLogicalSlotFuture() + .handle(assignResourceOrHandleError(deploymentHandle)); + slotAssignedFutures.add(slotAssigned); + } + return FutureUtils.waitForAll(slotAssignedFutures); + } + + private BiFunction deployAll(final Collection deploymentHandles) { + return (ignored, throwable) -> { + propagateIfNonNull(throwable); + for (final DeploymentHandle deploymentHandle : deploymentHandles) { + final SlotExecutionVertexAssignment slotExecutionVertexAssignment = deploymentHandle.getSlotExecutionVertexAssignment(); + final CompletableFuture slotAssigned = slotExecutionVertexAssignment.getLogicalSlotFuture(); + checkState(slotAssigned.isDone()); + + FutureUtils.assertNoException( + slotAssigned.handle(deployOrHandleError(deploymentHandle))); + } + return null; + }; + } + + private static void propagateIfNonNull(final Throwable throwable) { + if (throwable != null) { + throw new CompletionException(throwable); + } + } + + private BiFunction assignResourceOrHandleError(final DeploymentHandle deploymentHandle) { + final ExecutionVertexVersion requiredVertexVersion = deploymentHandle.getRequiredVertexVersion(); + final ExecutionVertexID executionVertexId = deploymentHandle.getExecutionVertexId(); + + return (logicalSlot, throwable) -> { + if (executionVertexVersioner.isModified(requiredVertexVersion)) { + log.debug("Refusing to assign slot to execution vertex {} because this deployment was " + + "superseded by another deployment", executionVertexId); + stopDeployment(deploymentHandle); + return null; + } + + if (throwable == null) { + final ExecutionVertex executionVertex = getExecutionVertex(executionVertexId); + final boolean sendScheduleOrUpdateConsumerMessage = deploymentHandle.getDeploymentOption().sendScheduleOrUpdateConsumerMessage(); + executionVertex + .getCurrentExecutionAttempt() + .registerProducedPartitions(logicalSlot.getTaskManagerLocation(), sendScheduleOrUpdateConsumerMessage); + executionVertex.tryAssignResource(logicalSlot); + } else { + handleTaskFailure(executionVertexId, throwable); + } + return null; + }; + } + + private BiFunction deployOrHandleError(final DeploymentHandle deploymentHandle) { + final ExecutionVertexVersion requiredVertexVersion = deploymentHandle.getRequiredVertexVersion(); + final ExecutionVertexID executionVertexId = requiredVertexVersion.getExecutionVertexId(); + + return (ignored, throwable) -> { + if (executionVertexVersioner.isModified(requiredVertexVersion)) { + log.debug("Refusing to deploy execution vertex {} because this deployment was " + + "superseded by another deployment", executionVertexId); + stopDeployment(deploymentHandle); + return null; + } + + if (throwable == null) { + deployTaskSafe(executionVertexId); + } else { + handleTaskFailure(executionVertexId, throwable); + } + return null; + }; + } + + private void stopDeployment(final DeploymentHandle deploymentHandle) { + cancelExecutionVertex(deploymentHandle.getExecutionVertexId()); + // Canceling the vertex normally releases the slot. However, we might not have assigned + // the slot to the vertex yet. + deploymentHandle + .getLogicalSlot() + .ifPresent(logicalSlot -> logicalSlot.releaseSlot(null)); + } + + private void deployTaskSafe(final ExecutionVertexID executionVertexId) { + try { + final ExecutionVertex executionVertex = getExecutionVertex(executionVertexId); + executionVertexOperations.deploy(executionVertex); + } catch (Throwable e) { + handleTaskFailure(executionVertexId, e); + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java index 5779a7ed08..0e663f39bc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java @@ -23,11 +23,19 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobWriter; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; +import org.apache.flink.runtime.executiongraph.failover.flip1.NoRestartBackoffTimeStrategy; +import org.apache.flink.runtime.executiongraph.failover.flip1.RestartBackoffTimeStrategy; +import org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy; import org.apache.flink.runtime.io.network.partition.PartitionTracker; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker; +import org.apache.flink.runtime.scheduler.strategy.EagerSchedulingStrategy; +import org.apache.flink.runtime.scheduler.strategy.LazyFromSourcesSchedulingStrategy; +import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategyFactory; import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.slf4j.Logger; @@ -58,6 +66,9 @@ public class DefaultSchedulerFactory implements SchedulerNGFactory { final ShuffleMaster shuffleMaster, final PartitionTracker partitionTracker) throws Exception { + final SchedulingStrategyFactory schedulingStrategyFactory = createSchedulingStrategyFactory(jobGraph.getScheduleMode()); + final RestartBackoffTimeStrategy restartBackoffTimeStrategy = NoRestartBackoffTimeStrategy.INSTANCE; + return new DefaultScheduler( log, jobGraph, @@ -66,6 +77,7 @@ public class DefaultSchedulerFactory implements SchedulerNGFactory { jobMasterConfiguration, slotProvider, futureExecutor, + new ScheduledExecutorServiceAdapter(futureExecutor), userCodeLoader, checkpointRecoveryFactory, rpcTimeout, @@ -73,7 +85,22 @@ public class DefaultSchedulerFactory implements SchedulerNGFactory { jobManagerJobMetricGroup, slotRequestTimeout, shuffleMaster, - partitionTracker); + partitionTracker, + schedulingStrategyFactory, + new RestartPipelinedRegionStrategy.Factory(), + restartBackoffTimeStrategy, + new DefaultExecutionVertexOperations(), + new ExecutionVertexVersioner()); } + private SchedulingStrategyFactory createSchedulingStrategyFactory(final ScheduleMode scheduleMode) { + switch (scheduleMode) { + case EAGER: + return new EagerSchedulingStrategy.Factory(); + case LAZY_FROM_SOURCES: + return new LazyFromSourcesSchedulingStrategy.Factory(); + default: + throw new IllegalStateException("Unsupported schedule mode " + scheduleMode); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DeploymentHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DeploymentHandle.java new file mode 100644 index 0000000000..2aed7fb312 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DeploymentHandle.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.runtime.scheduler; + +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.util.Preconditions; + +import java.util.Optional; +import java.util.concurrent.CompletableFuture; + +/** + * This class is a tuple holding the information necessary to deploy an {@link ExecutionVertex}. + * + *

The tuple consists of: + *

    + *
  • {@link ExecutionVertexVersion} + *
  • {@link ExecutionVertexDeploymentOption} + *
  • {@link SlotExecutionVertexAssignment} + *
+ */ +class DeploymentHandle { + + private final ExecutionVertexVersion requiredVertexVersion; + + private final ExecutionVertexDeploymentOption executionVertexDeploymentOption; + + private final SlotExecutionVertexAssignment slotExecutionVertexAssignment; + + public DeploymentHandle( + final ExecutionVertexVersion requiredVertexVersion, + final ExecutionVertexDeploymentOption executionVertexDeploymentOption, + final SlotExecutionVertexAssignment slotExecutionVertexAssignment) { + + this.requiredVertexVersion = Preconditions.checkNotNull(requiredVertexVersion); + this.executionVertexDeploymentOption = Preconditions.checkNotNull(executionVertexDeploymentOption); + this.slotExecutionVertexAssignment = Preconditions.checkNotNull(slotExecutionVertexAssignment); + } + + public ExecutionVertexID getExecutionVertexId() { + return requiredVertexVersion.getExecutionVertexId(); + } + + public ExecutionVertexVersion getRequiredVertexVersion() { + return requiredVertexVersion; + } + + public DeploymentOption getDeploymentOption() { + return executionVertexDeploymentOption.getDeploymentOption(); + } + + public SlotExecutionVertexAssignment getSlotExecutionVertexAssignment() { + return slotExecutionVertexAssignment; + } + + public Optional getLogicalSlot() { + final CompletableFuture logicalSlotFuture = slotExecutionVertexAssignment.getLogicalSlotFuture(); + Preconditions.checkState(logicalSlotFuture.isDone(), "method can only be called after slot future is done"); + + if (logicalSlotFuture.isCompletedExceptionally() || logicalSlotFuture.isCancelled()) { + return Optional.empty(); + } + return Optional.ofNullable(logicalSlotFuture.getNow(null)); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionVertexOperations.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionVertexOperations.java new file mode 100644 index 0000000000..23cca7c709 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionVertexOperations.java @@ -0,0 +1,35 @@ +/* + * 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.scheduler; + +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; + +import java.util.concurrent.CompletableFuture; + +/** + * Operations on the {@link ExecutionVertex}. + */ +interface ExecutionVertexOperations { + + void deploy(ExecutionVertex executionVertex) throws JobException; + + CompletableFuture cancel(ExecutionVertex executionVertex); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionVertexSchedulingRequirementsMapper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionVertexSchedulingRequirementsMapper.java new file mode 100644 index 0000000000..b089e5fe1f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionVertexSchedulingRequirementsMapper.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.runtime.scheduler; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +import java.util.Collection; +import java.util.Collections; + +final class ExecutionVertexSchedulingRequirementsMapper { + + public static ExecutionVertexSchedulingRequirements from(final ExecutionVertex executionVertex) { + + final ExecutionVertexID executionVertexId = executionVertex.getID(); + + final AllocationID latestPriorAllocation = executionVertex.getLatestPriorAllocation(); + final SlotSharingGroup slotSharingGroup = executionVertex.getJobVertex().getSlotSharingGroup(); + + return new ExecutionVertexSchedulingRequirements.Builder() + .withExecutionVertexId(executionVertexId) + .withPreviousAllocationId(latestPriorAllocation) + .withResourceProfile(executionVertex.getResourceProfile()) + .withSlotSharingGroupId(slotSharingGroup == null ? null : slotSharingGroup.getSlotSharingGroupId()) + .withCoLocationConstraint(executionVertex.getLocationConstraint()) + .withPreferredLocations(getPreferredLocationBasedOnState(executionVertex)).build(); + } + + private static Collection getPreferredLocationBasedOnState(final ExecutionVertex executionVertex) { + return executionVertex + .getPreferredLocationBasedOnState() + .map(Collections::singleton) + .orElse(Collections.emptySet()); + } + + private ExecutionVertexSchedulingRequirementsMapper() { + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/InternalTaskFailuresListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/InternalTaskFailuresListener.java new file mode 100644 index 0000000000..6dbda3bfb1 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/InternalTaskFailuresListener.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.runtime.scheduler; + +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; + +/** + * This interface enables subscribing to Task failures that are detected from the JobMaster side + * (e.g., from within the {@link ExecutionGraph}). + * In contrast, there are also failures that are detected by the TaskManager, which are communicated + * via {@link JobMasterGateway#updateTaskExecutionState(TaskExecutionState)}. + */ +public interface InternalTaskFailuresListener { + + void notifyFailed(ExecutionAttemptID attemptId, Throwable t); + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/LegacyScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/LegacyScheduler.java index bc6ed69be0..bc5c45091f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/LegacyScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/LegacyScheduler.java @@ -19,78 +19,24 @@ package org.apache.flink.runtime.scheduler; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; -import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.io.InputSplit; -import org.apache.flink.queryablestate.KvStateID; -import org.apache.flink.runtime.JobException; -import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.blob.BlobWriter; -import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; -import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; -import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; -import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; -import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; -import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; -import org.apache.flink.runtime.executiongraph.Execution; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionGraph; -import org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder; -import org.apache.flink.runtime.executiongraph.ExecutionGraphException; -import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; -import org.apache.flink.runtime.executiongraph.IntermediateResult; -import org.apache.flink.runtime.executiongraph.JobStatusListener; -import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; -import org.apache.flink.runtime.executiongraph.restart.RestartStrategyResolving; import org.apache.flink.runtime.io.network.partition.PartitionTracker; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.runtime.jobmanager.PartitionProducerDisposedException; -import org.apache.flink.runtime.jobmaster.SerializedInputSplit; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; -import org.apache.flink.runtime.messages.FlinkJobNotFoundException; -import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; -import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; -import org.apache.flink.runtime.messages.webmonitor.JobDetails; import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; -import org.apache.flink.runtime.query.KvStateLocation; -import org.apache.flink.runtime.query.KvStateLocationRegistry; -import org.apache.flink.runtime.query.UnknownKvStateLocation; import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker; -import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStats; import org.apache.flink.runtime.shuffle.ShuffleMaster; -import org.apache.flink.runtime.state.KeyGroupRange; -import org.apache.flink.runtime.taskmanager.TaskExecutionState; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.apache.flink.runtime.webmonitor.WebMonitorUtils; -import org.apache.flink.util.FlinkException; -import org.apache.flink.util.InstantiationUtil; -import org.apache.flink.util.function.FunctionUtils; import org.slf4j.Logger; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; -import static org.apache.flink.util.Preconditions.checkNotNull; - /** * A scheduler that delegates to the scheduling logic in the {@link ExecutionGraph}. * @@ -134,4 +80,15 @@ public class LegacyScheduler extends SchedulerBase { shuffleMaster, partitionTracker); } + + @Override + protected void startSchedulingInternal() { + final ExecutionGraph executionGraph = getExecutionGraph(); + try { + executionGraph.scheduleForExecution(); + } + catch (Throwable t) { + executionGraph.failGlobal(t); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java index 6c2ed517e1..4051ed4f32 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java @@ -45,8 +45,11 @@ import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder; import org.apache.flink.runtime.executiongraph.ExecutionGraphException; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.executiongraph.IntermediateResult; import org.apache.flink.runtime.executiongraph.JobStatusListener; +import org.apache.flink.runtime.executiongraph.failover.adapter.DefaultFailoverTopology; +import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyResolving; @@ -70,6 +73,9 @@ import org.apache.flink.runtime.query.KvStateLocationRegistry; import org.apache.flink.runtime.query.UnknownKvStateLocation; import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker; import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStats; +import org.apache.flink.runtime.scheduler.adapter.ExecutionGraphToSchedulingTopologyAdapter; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.taskmanager.TaskExecutionState; @@ -83,6 +89,7 @@ import org.slf4j.Logger; import java.io.IOException; import java.net.InetSocketAddress; +import java.util.Collection; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -102,6 +109,12 @@ public abstract class SchedulerBase implements SchedulerNG { private final ExecutionGraph executionGraph; + private final SchedulingTopology schedulingTopology; + + private final FailoverTopology failoverTopology; + + private final InputsLocationsRetriever inputsLocationsRetriever; + private final BackPressureStatsTracker backPressureStatsTracker; private final Executor ioExecutor; @@ -124,9 +137,10 @@ public abstract class SchedulerBase implements SchedulerNG { private final Time slotRequestTimeout; + private ComponentMainThreadExecutor mainThreadExecutor = new ComponentMainThreadExecutor.DummyComponentMainThreadExecutor( - "LegacyScheduler is not initialized with proper main thread executor. " + - "Call to LegacyScheduler.setMainThreadExecutor(...) required."); + "SchedulerBase is not initialized with proper main thread executor. " + + "Call to SchedulerBase.setMainThreadExecutor(...) required."); public SchedulerBase( final Logger log, @@ -172,6 +186,9 @@ public abstract class SchedulerBase implements SchedulerNG { this.slotRequestTimeout = checkNotNull(slotRequestTimeout); this.executionGraph = createAndRestoreExecutionGraph(jobManagerJobMetricGroup, checkNotNull(shuffleMaster), checkNotNull(partitionTracker)); + this.schedulingTopology = new ExecutionGraphToSchedulingTopologyAdapter(executionGraph); + this.failoverTopology = new DefaultFailoverTopology(executionGraph); + this.inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(executionGraph); } private ExecutionGraph createAndRestoreExecutionGraph( @@ -221,6 +238,26 @@ public abstract class SchedulerBase implements SchedulerNG { partitionTracker); } + /** + * @deprecated Direct access to the execution graph by scheduler implementations is discouraged + * because currently the execution graph has various features and responsibilities that a + * scheduler should not be concerned about. The following specialized abstractions to the + * execution graph and accessors should be preferred over direct access: + *
    + *
  • {@link #getSchedulingTopology()} + *
  • {@link #getFailoverTopology()} + *
  • {@link #getInputsLocationsRetriever()} + *
  • {@link #getExecutionVertex(ExecutionVertexID)} + *
  • {@link #getExecutionVertexId(ExecutionAttemptID)} + *
  • {@link #getExecutionVertexIdOrThrow(ExecutionAttemptID)} + *
+ * Currently, only {@link LegacyScheduler} requires direct access to the execution graph. + */ + @Deprecated + protected ExecutionGraph getExecutionGraph() { + return executionGraph; + } + /** * Tries to restore the given {@link ExecutionGraph} from the provided {@link SavepointRestoreSettings}. * @@ -241,6 +278,68 @@ public abstract class SchedulerBase implements SchedulerNG { } } + protected void resetForNewExecutionIfInTerminalState(final Collection verticesToDeploy) { + verticesToDeploy.forEach(executionVertexId -> getExecutionVertex(executionVertexId) + .resetForNewExecutionIfInTerminalState()); + } + + protected void transitionToScheduled(final Collection verticesToDeploy) { + verticesToDeploy.forEach(executionVertexId -> getExecutionVertex(executionVertexId) + .getCurrentExecutionAttempt() + .transitionState(ExecutionState.SCHEDULED)); + } + + protected ComponentMainThreadExecutor getMainThreadExecutor() { + return mainThreadExecutor; + } + + protected void failJob(Throwable cause) { + executionGraph.failJob(cause); + } + + protected FailoverTopology getFailoverTopology() { + return failoverTopology; + } + + protected SchedulingTopology getSchedulingTopology() { + return schedulingTopology; + } + + protected InputsLocationsRetriever getInputsLocationsRetriever() { + return inputsLocationsRetriever; + } + + protected final void prepareExecutionGraphForNgScheduling() { + executionGraph.enableNgScheduling(new UpdateSchedulerNgOnInternalTaskFailuresListener(this, jobGraph.getJobID())); + executionGraph.transitionToRunning(); + } + + protected Optional getExecutionVertexId(final ExecutionAttemptID executionAttemptId) { + return Optional.ofNullable(executionGraph.getRegisteredExecutions().get(executionAttemptId)) + .map(this::getExecutionVertexId); + } + + protected ExecutionVertexID getExecutionVertexIdOrThrow(final ExecutionAttemptID executionAttemptId) { + return getExecutionVertexId(executionAttemptId) + .orElseThrow(() -> new IllegalStateException("Cannot find execution " + executionAttemptId)); + } + + private ExecutionVertexID getExecutionVertexId(final Execution execution) { + return execution.getVertex().getID(); + } + + protected ExecutionVertex getExecutionVertex(final ExecutionVertexID executionVertexId) { + return executionGraph.getAllVertices().get(executionVertexId.getJobVertexId()).getTaskVertices()[executionVertexId.getSubtaskIndex()]; + } + + protected JobGraph getJobGraph() { + return jobGraph; + } + + // ------------------------------------------------------------------------ + // SchedulerNG + // ------------------------------------------------------------------------ + @Override public void setMainThreadExecutor(final ComponentMainThreadExecutor mainThreadExecutor) { this.mainThreadExecutor = checkNotNull(mainThreadExecutor); @@ -253,17 +352,13 @@ public abstract class SchedulerBase implements SchedulerNG { } @Override - public void startScheduling() { + public final void startScheduling() { mainThreadExecutor.assertRunningInMainThread(); - - try { - executionGraph.scheduleForExecution(); - } - catch (Throwable t) { - executionGraph.failGlobal(t); - } + startSchedulingInternal(); } + protected abstract void startSchedulingInternal(); + @Override public void suspend(Throwable cause) { mainThreadExecutor.assertRunningInMainThread(); @@ -282,9 +377,17 @@ public abstract class SchedulerBase implements SchedulerNG { } @Override - public boolean updateTaskExecutionState(final TaskExecutionState taskExecutionState) { - mainThreadExecutor.assertRunningInMainThread(); - return executionGraph.updateState(taskExecutionState); + public final boolean updateTaskExecutionState(final TaskExecutionState taskExecutionState) { + final Optional executionVertexId = getExecutionVertexId(taskExecutionState.getID()); + if (executionVertexId.isPresent()) { + executionGraph.updateState(taskExecutionState); + updateTaskExecutionStateInternal(executionVertexId.get(), taskExecutionState); + return true; + } + return false; + } + + protected void updateTaskExecutionStateInternal(final ExecutionVertexID executionVertexId, final TaskExecutionState taskExecutionState) { } @Override @@ -363,14 +466,20 @@ public abstract class SchedulerBase implements SchedulerNG { } @Override - public void scheduleOrUpdateConsumers(final ResultPartitionID partitionID) { + public final void scheduleOrUpdateConsumers(final ResultPartitionID partitionId) { mainThreadExecutor.assertRunningInMainThread(); try { - executionGraph.scheduleOrUpdateConsumers(partitionID); + executionGraph.scheduleOrUpdateConsumers(partitionId); } catch (ExecutionGraphException e) { throw new RuntimeException(e); } + + final ExecutionVertexID producerVertexId = getExecutionVertexIdOrThrow(partitionId.getProducerId()); + scheduleOrUpdateConsumersInternal(producerVertexId, partitionId); + } + + protected void scheduleOrUpdateConsumersInternal(ExecutionVertexID producerVertexId, ResultPartitionID resultPartitionId) { } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/UpdateSchedulerNgOnInternalTaskFailuresListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/UpdateSchedulerNgOnInternalTaskFailuresListener.java new file mode 100644 index 0000000000..f37e4ecddb --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/UpdateSchedulerNgOnInternalTaskFailuresListener.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.runtime.scheduler; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Calls {@link SchedulerNG#updateTaskExecutionState(TaskExecutionState)} on task failure. + */ +class UpdateSchedulerNgOnInternalTaskFailuresListener implements InternalTaskFailuresListener { + + private final SchedulerNG schedulerNg; + + private final JobID jobId; + + public UpdateSchedulerNgOnInternalTaskFailuresListener( + final SchedulerNG schedulerNg, + final JobID jobId) { + + this.schedulerNg = checkNotNull(schedulerNg); + this.jobId = checkNotNull(jobId); + } + + @Override + public void notifyFailed(final ExecutionAttemptID attemptId, final Throwable t) { + schedulerNg.updateTaskExecutionState(new TaskExecutionState( + jobId, + attemptId, + ExecutionState.FAILED, + t)); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java index 6093000cbe..f0a4524e4a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java @@ -314,7 +314,7 @@ public class ExecutionVertexDeploymentTest extends TestLogger { new AllocationID(), 0, null, - Execution.registerProducedPartitions(vertex, location, attemptID).get().values()); + Execution.registerProducedPartitions(vertex, location, attemptID, scheduleMode.allowLazyDeployment()).get().values()); Collection producedPartitions = tdd.getProducedPartitions(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/TestRestartBackoffTimeStrategy.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/TestRestartBackoffTimeStrategy.java index 878c89f4c5..0606cce3ee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/TestRestartBackoffTimeStrategy.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/TestRestartBackoffTimeStrategy.java @@ -24,9 +24,9 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; */ public class TestRestartBackoffTimeStrategy implements RestartBackoffTimeStrategy { - private final boolean canRestart; + private boolean canRestart; - private final long backoffTime; + private long backoffTime; public TestRestartBackoffTimeStrategy(boolean canRestart, long backoffTime) { this.canRestart = canRestart; @@ -47,4 +47,12 @@ public class TestRestartBackoffTimeStrategy implements RestartBackoffTimeStrateg public void notifyFailure(Throwable cause) { // ignore } + + public void setCanRestart(final boolean canRestart) { + this.canRestart = canRestart; + } + + public void setBackoffTime(final long backoffTime) { + this.backoffTime = backoffTime; + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java new file mode 100644 index 0000000000..7988e73a97 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java @@ -0,0 +1,316 @@ +/* + * 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.scheduler; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.runtime.blob.VoidBlobWriter; +import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; +import org.apache.flink.runtime.clusterframework.types.SlotProfile; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionVertex; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.failover.FailoverStrategyLoader; +import org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy; +import org.apache.flink.runtime.executiongraph.failover.flip1.TestRestartBackoffTimeStrategy; +import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider; +import org.apache.flink.runtime.io.network.partition.NoOpPartitionTracker; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.ScheduleMode; +import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.rest.handler.legacy.backpressure.VoidBackPressureStatsTracker; +import org.apache.flink.runtime.scheduler.strategy.EagerSchedulingStrategy; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.LazyFromSourcesSchedulingStrategy; +import org.apache.flink.runtime.shuffle.NettyShuffleMaster; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; +import org.apache.flink.util.ExecutorUtils; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; + +import org.hamcrest.Matchers; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link DefaultScheduler}. + */ +public class DefaultSchedulerTest extends TestLogger { + + private static final int TIMEOUT_MS = 1000; + + private static final JobID TEST_JOB_ID = new JobID(); + + private ManuallyTriggeredScheduledExecutor taskRestartExecutor = new ManuallyTriggeredScheduledExecutor(); + + private ExecutorService executor; + + private ScheduledExecutorService scheduledExecutorService; + + private Configuration configuration; + + private SubmissionTrackingTaskManagerGateway testTaskManagerGateway; + + private TestRestartBackoffTimeStrategy testRestartBackoffTimeStrategy; + + private FailingExecutionVertexOperationsDecorator testExecutionVertexOperations; + + private SimpleSlotProvider slotProvider; + + private ExecutionVertexVersioner executionVertexVersioner; + + @Before + public void setUp() throws Exception { + executor = Executors.newSingleThreadExecutor(); + scheduledExecutorService = new DirectScheduledExecutorService(); + + configuration = new Configuration(); + configuration.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, FailoverStrategyLoader.NO_OP_FAILOVER_STRATEGY); + testTaskManagerGateway = new SubmissionTrackingTaskManagerGateway(); + + testRestartBackoffTimeStrategy = new TestRestartBackoffTimeStrategy(true, 0); + + testExecutionVertexOperations = new FailingExecutionVertexOperationsDecorator(new DefaultExecutionVertexOperations()); + + slotProvider = new SimpleSlotProvider(TEST_JOB_ID, 12, testTaskManagerGateway); + + executionVertexVersioner = new ExecutionVertexVersioner(); + } + + @After + public void tearDown() throws Exception { + if (scheduledExecutorService != null) { + ExecutorUtils.gracefulShutdown(TIMEOUT_MS, TimeUnit.MILLISECONDS, scheduledExecutorService); + } + + if (executor != null) { + ExecutorUtils.gracefulShutdown(TIMEOUT_MS, TimeUnit.MILLISECONDS, executor); + } + } + + @Test + public void startScheduling() { + final JobGraph jobGraph = singleNonParallelJobVertexJobGraph(); + final JobVertex onlyJobVertex = getOnlyJobVertex(jobGraph); + + createSchedulerAndStartScheduling(jobGraph); + + final List deployedExecutionVertices = testTaskManagerGateway.getDeployedExecutionVertices(1, TIMEOUT_MS); + + final ExecutionVertexID executionVertexId = new ExecutionVertexID(onlyJobVertex.getID(), 0); + assertThat(deployedExecutionVertices, contains(executionVertexId)); + } + + @Test + public void restartAfterDeploymentFails() { + final JobGraph jobGraph = singleNonParallelJobVertexJobGraph(); + final JobVertex onlyJobVertex = getOnlyJobVertex(jobGraph); + + testExecutionVertexOperations.enableFailDeploy(); + + createSchedulerAndStartScheduling(jobGraph); + + testExecutionVertexOperations.disableFailDeploy(); + taskRestartExecutor.triggerScheduledTasks(); + + final List deployedExecutionVertices = testTaskManagerGateway.getDeployedExecutionVertices(1, TIMEOUT_MS); + + final ExecutionVertexID executionVertexId = new ExecutionVertexID(onlyJobVertex.getID(), 0); + assertThat(deployedExecutionVertices, contains(executionVertexId)); + } + + @Test + public void scheduleWithLazyStrategy() { + final JobGraph jobGraph = singleNonParallelJobVertexJobGraph(); + jobGraph.setScheduleMode(ScheduleMode.LAZY_FROM_SOURCES); + final JobVertex onlyJobVertex = getOnlyJobVertex(jobGraph); + + createSchedulerAndStartScheduling(jobGraph); + + final List deployedExecutionVertices = testTaskManagerGateway.getDeployedExecutionVertices(1, TIMEOUT_MS); + + final ExecutionVertexID executionVertexId = new ExecutionVertexID(onlyJobVertex.getID(), 0); + assertThat(deployedExecutionVertices, contains(executionVertexId)); + } + + @Test + public void restartFailedTask() { + final JobGraph jobGraph = singleNonParallelJobVertexJobGraph(); + final JobVertex onlyJobVertex = getOnlyJobVertex(jobGraph); + + final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph); + + final ArchivedExecutionVertex archivedExecutionVertex = Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices()); + final ExecutionAttemptID attemptId = archivedExecutionVertex.getCurrentExecutionAttempt().getAttemptId(); + + scheduler.updateTaskExecutionState(new TaskExecutionState(jobGraph.getJobID(), attemptId, ExecutionState.FAILED)); + + taskRestartExecutor.triggerScheduledTasks(); + + final List deployedExecutionVertices = testTaskManagerGateway.getDeployedExecutionVertices(2, TIMEOUT_MS); + final ExecutionVertexID executionVertexId = new ExecutionVertexID(onlyJobVertex.getID(), 0); + assertThat(deployedExecutionVertices, contains(executionVertexId, executionVertexId)); + } + + @Test + public void updateTaskExecutionStateReturnsFalseIfExecutionDoesNotExist() { + final JobGraph jobGraph = singleNonParallelJobVertexJobGraph(); + final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph); + + final TaskExecutionState taskExecutionState = new TaskExecutionState( + jobGraph.getJobID(), + new ExecutionAttemptID(), + ExecutionState.FAILED); + + assertFalse(scheduler.updateTaskExecutionState(taskExecutionState)); + } + + @Test + public void failJobIfCannotRestart() throws Exception { + final JobGraph jobGraph = singleNonParallelJobVertexJobGraph(); + testRestartBackoffTimeStrategy.setCanRestart(false); + + final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph); + + final ArchivedExecutionVertex onlyExecutionVertex = Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices()); + final ExecutionAttemptID attemptId = onlyExecutionVertex.getCurrentExecutionAttempt().getAttemptId(); + + scheduler.updateTaskExecutionState(new TaskExecutionState(jobGraph.getJobID(), attemptId, ExecutionState.FAILED)); + + taskRestartExecutor.triggerScheduledTasks(); + + waitForTermination(scheduler); + final JobStatus jobStatus = scheduler.requestJobStatus(); + assertThat(jobStatus, is(Matchers.equalTo(JobStatus.FAILED))); + } + + @Test + public void failJobIfNotEnoughResources() throws Exception { + drainAllAvailableSlots(); + + final JobGraph jobGraph = singleNonParallelJobVertexJobGraph(); + testRestartBackoffTimeStrategy.setCanRestart(false); + + final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph); + + waitForTermination(scheduler); + final JobStatus jobStatus = scheduler.requestJobStatus(); + assertThat(jobStatus, is(Matchers.equalTo(JobStatus.FAILED))); + } + + private void drainAllAvailableSlots() { + final int numberOfAvailableSlots = slotProvider.getNumberOfAvailableSlots(); + for (int i = 0; i < numberOfAvailableSlots; i++) { + slotProvider.allocateSlot( + new SlotRequestId(), + new ScheduledUnit(new JobVertexID(), null, null), + SlotProfile.noRequirements(), + true, + Time.milliseconds(TIMEOUT_MS)); + } + } + + private void waitForTermination(final DefaultScheduler scheduler) throws Exception { + scheduler.getTerminationFuture().get(TIMEOUT_MS, TimeUnit.MILLISECONDS); + } + + private static JobGraph singleNonParallelJobVertexJobGraph() { + final JobGraph jobGraph = new JobGraph(TEST_JOB_ID, "Testjob"); + jobGraph.setScheduleMode(ScheduleMode.EAGER); + final JobVertex vertex = new JobVertex("source"); + vertex.setInvokableClass(NoOpInvokable.class); + jobGraph.addVertex(vertex); + return jobGraph; + } + + private static JobVertex getOnlyJobVertex(final JobGraph jobGraph) { + final List sortedVertices = jobGraph.getVerticesSortedTopologicallyFromSources(); + Preconditions.checkState(sortedVertices.size() == 1); + return sortedVertices.get(0); + } + + private DefaultScheduler createSchedulerAndStartScheduling(final JobGraph jobGraph) { + try { + final DefaultScheduler scheduler = createScheduler(jobGraph); + startScheduling(scheduler); + return scheduler; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private DefaultScheduler createScheduler(final JobGraph jobGraph) throws Exception { + return new DefaultScheduler( + log, + jobGraph, + VoidBackPressureStatsTracker.INSTANCE, + executor, + configuration, + slotProvider, + scheduledExecutorService, + taskRestartExecutor, + ClassLoader.getSystemClassLoader(), + new StandaloneCheckpointRecoveryFactory(), + Time.seconds(300), + VoidBlobWriter.getInstance(), + UnregisteredMetricGroups.createUnregisteredJobManagerJobMetricGroup(), + Time.seconds(300), + NettyShuffleMaster.INSTANCE, + NoOpPartitionTracker.INSTANCE, + jobGraph.getScheduleMode() == ScheduleMode.LAZY_FROM_SOURCES ? + new LazyFromSourcesSchedulingStrategy.Factory() : + new EagerSchedulingStrategy.Factory(), + new RestartPipelinedRegionStrategy.Factory(), + testRestartBackoffTimeStrategy, + testExecutionVertexOperations, + executionVertexVersioner); + } + + private void startScheduling(final SchedulerNG scheduler) { + scheduler.setMainThreadExecutor(ComponentMainThreadExecutorServiceAdapter.forMainThread()); + scheduler.startScheduling(); + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/FailingExecutionVertexOperationsDecorator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/FailingExecutionVertexOperationsDecorator.java new file mode 100644 index 0000000000..b5ad29f96b --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/FailingExecutionVertexOperationsDecorator.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.runtime.scheduler; + +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; + +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Allows to fail ExecutionVertex operations for testing. + */ +public class FailingExecutionVertexOperationsDecorator implements ExecutionVertexOperations { + + private final ExecutionVertexOperations delegate; + + private boolean failDeploy; + + private boolean failCancel; + + public FailingExecutionVertexOperationsDecorator(final ExecutionVertexOperations delegate) { + this.delegate = checkNotNull(delegate); + } + + @Override + public void deploy(final ExecutionVertex executionVertex) throws JobException { + if (failDeploy) { + throw new RuntimeException("Expected"); + } else { + delegate.deploy(executionVertex); + } + } + + @Override + public CompletableFuture cancel(final ExecutionVertex executionVertex) { + if (failCancel) { + return FutureUtils.completedExceptionally(new RuntimeException("Expected")); + } else { + return delegate.cancel(executionVertex); + } + } + + public void enableFailDeploy() { + failDeploy = true; + } + + public void disableFailDeploy() { + failDeploy = false; + } + + public void enableFailCancel() { + failCancel = true; + } + + public void disableFailCancel() { + failCancel = false; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SubmissionTrackingTaskManagerGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SubmissionTrackingTaskManagerGateway.java new file mode 100644 index 0000000000..ae1c01ba50 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SubmissionTrackingTaskManagerGateway.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.runtime.scheduler; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.executiongraph.TaskInformation; +import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.util.Preconditions.checkState; + +class SubmissionTrackingTaskManagerGateway extends SimpleAckingTaskManagerGateway { + + private final BlockingQueue taskDeploymentDescriptors = new LinkedBlockingDeque<>(); + + private boolean failSubmission; + + public void setFailSubmission(final boolean failSubmission) { + this.failSubmission = failSubmission; + } + + @Override + public CompletableFuture submitTask(final TaskDeploymentDescriptor tdd, final Time timeout) { + super.submitTask(tdd, timeout); + + taskDeploymentDescriptors.add(tdd); + + if (failSubmission) { + return FutureUtils.completedExceptionally(new RuntimeException("Task submission failed.")); + } else { + return CompletableFuture.completedFuture(Acknowledge.get()); + } + } + + public List getDeployedExecutionVertices(int num, long timeoutMs) { + final List deployedVertices = new ArrayList<>(); + for (int i = 0; i < num; i++) { + try { + final TaskDeploymentDescriptor taskDeploymentDescriptor = taskDeploymentDescriptors.poll(timeoutMs, TimeUnit.MILLISECONDS); + checkState(taskDeploymentDescriptor != null, "Expected %s tasks to be submitted within %s ms, got %s", num, timeoutMs, i); + deployedVertices.add(getExecutionVertexId(taskDeploymentDescriptor)); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + return deployedVertices; + } + + private ExecutionVertexID getExecutionVertexId(final TaskDeploymentDescriptor deploymentDescriptor) { + final TaskInformation taskInformation = deserializeTaskInformation(deploymentDescriptor); + final JobVertexID jobVertexId = taskInformation.getJobVertexId(); + final int subtaskIndex = deploymentDescriptor.getSubtaskIndex(); + return new ExecutionVertexID(jobVertexId, subtaskIndex); + } + + private TaskInformation deserializeTaskInformation(final TaskDeploymentDescriptor taskDeploymentDescriptor) { + try { + return taskDeploymentDescriptor + .getSerializedTaskInformation() + .deserializeValue(ClassLoader.getSystemClassLoader()); + } catch (IOException | ClassNotFoundException e) { + throw new RuntimeException(e); + } + } +} -- Gitee From b8d491c97d305fed7ac680929bb04dd714657c7c Mon Sep 17 00:00:00 2001 From: a-suiniaev Date: Wed, 25 Sep 2019 03:26:27 -0400 Subject: [PATCH 060/268] [FLINK-14120][API/Datastream] Fix SystemProcessingTimeServiceTest.testImmediateShutdown Fix testImmediateShutdown test: delegated thread synchronization to completableFuture Added more explicit timeout for awaiting future completion This closes #9767. --- .../SystemProcessingTimeServiceTest.java | 60 +++++++++++-------- 1 file changed, 34 insertions(+), 26 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeServiceTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeServiceTest.java index a01a9c8fc8..493dbf3636 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeServiceTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeServiceTest.java @@ -26,6 +26,7 @@ import org.junit.Assert; import org.junit.Test; import java.util.concurrent.CancellationException; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; @@ -34,10 +35,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; -import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -140,40 +138,26 @@ public class SystemProcessingTimeServiceTest extends TestLogger { @Test public void testImmediateShutdown() throws Exception { + final CompletableFuture errorFuture = new CompletableFuture<>(); - final Object lock = new Object(); - final AtomicReference errorRef = new AtomicReference<>(); - - final SystemProcessingTimeService timer = createSystemProcessingTimeService(errorRef); + final SystemProcessingTimeService timer = createSystemProcessingTimeService(errorFuture); try { assertFalse(timer.isTerminated()); final OneShotLatch latch = new OneShotLatch(); - // the task should trigger immediately and should block until terminated with interruption + // the task should trigger immediately and sleep until terminated with interruption timer.registerTimer(System.currentTimeMillis(), timestamp -> { - synchronized (lock) { - latch.trigger(); - Thread.sleep(100000000); - } + latch.trigger(); + Thread.sleep(100000000); }); latch.await(); timer.shutdownService(); - synchronized (lock) { - assertTrue(timer.isTerminated()); - - // The shutdownService() may not necessary wait for active tasks to finish properly. - // From the ScheduledThreadPoolExecutor Java docs: - // There are no guarantees beyond best-effort attempts to stop processing actively executing tasks. - // This implementation cancels tasks via {@link Thread#interrupt}, so any task that - // fails to respond to interrupts may never terminate. - assertThat(errorRef.get(), is(anyOf(nullValue(), instanceOf(InterruptedException.class)))); - assertEquals(0, timer.getNumTasksScheduled()); - } - errorRef.set(null); + assertTrue(timer.isTerminated()); + assertEquals(0, timer.getNumTasksScheduled()); try { timer.registerTimer(System.currentTimeMillis() + 1000, timestamp -> fail("should not be called")); @@ -183,7 +167,6 @@ public class SystemProcessingTimeServiceTest extends TestLogger { catch (IllegalStateException e) { // expected } - assertThat(errorRef.get(), is(nullValue())); try { timer.scheduleAtFixedRate(timestamp -> fail("should not be called"), 0L, 100L); @@ -193,7 +176,9 @@ public class SystemProcessingTimeServiceTest extends TestLogger { catch (IllegalStateException e) { // expected } - assertThat(errorRef.get(), is(nullValue())); + + // check that the task eventually responded to interruption + assertThat(errorFuture.get(30L, TimeUnit.SECONDS), instanceOf(InterruptedException.class)); } finally { timer.shutdownService(); @@ -375,6 +360,12 @@ public class SystemProcessingTimeServiceTest extends TestLogger { Assert.assertTrue(timerFinished.get()); } + private static SystemProcessingTimeService createSystemProcessingTimeService(CompletableFuture errorFuture) { + Preconditions.checkArgument(!errorFuture.isDone()); + + return new SystemProcessingTimeService(new TestOnTimerCompletablyCallbackContext(errorFuture)); + } + private static SystemProcessingTimeService createSystemProcessingTimeService(AtomicReference errorRef) { Preconditions.checkArgument(errorRef.get() == null); @@ -439,4 +430,21 @@ public class SystemProcessingTimeServiceTest extends TestLogger { } } } + + private static class TestOnTimerCompletablyCallbackContext implements SystemProcessingTimeService.ScheduledCallbackExecutionContext { + private final CompletableFuture completableExceptionHandler; + + TestOnTimerCompletablyCallbackContext(CompletableFuture completableExceptionHandler) { + this.completableExceptionHandler = completableExceptionHandler; + } + + @Override + public void invoke(ProcessingTimeCallback callback, long timestamp) { + try { + callback.onProcessingTime(timestamp); + } catch (Throwable t) { + completableExceptionHandler.complete(t); + } + } + } } -- Gitee From 74f4a9f6f0b55b8569363ed95d4fa7dc379e0ce4 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Fri, 27 Sep 2019 15:21:05 +0200 Subject: [PATCH 061/268] [FLINK-14244][runtime] Initialize DefaultScheduler with correct RestartBackoffTimeStrategy This closes #9785. --- .../runtime/scheduler/DefaultSchedulerFactory.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java index 0e663f39bc..603e14d6e7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java @@ -24,8 +24,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobWriter; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; -import org.apache.flink.runtime.executiongraph.failover.flip1.NoRestartBackoffTimeStrategy; import org.apache.flink.runtime.executiongraph.failover.flip1.RestartBackoffTimeStrategy; +import org.apache.flink.runtime.executiongraph.failover.flip1.RestartBackoffTimeStrategyFactoryLoader; import org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy; import org.apache.flink.runtime.io.network.partition.PartitionTracker; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -67,7 +67,15 @@ public class DefaultSchedulerFactory implements SchedulerNGFactory { final PartitionTracker partitionTracker) throws Exception { final SchedulingStrategyFactory schedulingStrategyFactory = createSchedulingStrategyFactory(jobGraph.getScheduleMode()); - final RestartBackoffTimeStrategy restartBackoffTimeStrategy = NoRestartBackoffTimeStrategy.INSTANCE; + final RestartBackoffTimeStrategy restartBackoffTimeStrategy = RestartBackoffTimeStrategyFactoryLoader + .createRestartStrategyFactory( + jobGraph + .getSerializedExecutionConfig() + .deserializeValue(userCodeLoader) + .getRestartStrategy(), + jobMasterConfiguration, + jobGraph.isCheckpointingEnabled()) + .create(); return new DefaultScheduler( log, -- Gitee From 9e5d9704e72348fa54d79dd570d59ac80492a79a Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Mon, 23 Sep 2019 14:34:01 +0800 Subject: [PATCH 062/268] [hotfix][tests] Fix the invalid method reference in javadoc of unit tests --- .../io/network/partition/consumer/SingleInputGateTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index 057ac5ecc5..3a8e4f23ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -605,7 +605,7 @@ public class SingleInputGateTest extends InputGateTestBase { /** * Tests that if the {@link PartitionNotFoundException} is set onto one {@link InputChannel}, - * then it would be thrown directly via {@link SingleInputGate#getNextBufferOrEvent()}. So we + * then it would be thrown directly via {@link SingleInputGate#getNext()}. So we * could confirm the {@link SingleInputGate} would not swallow or transform the original exception. */ @Test -- Gitee From 4ccb0a1a1a66f00141cd534bfb3765e5d008d5c7 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Mon, 23 Sep 2019 14:36:17 +0800 Subject: [PATCH 063/268] [hotfix][tests] Remove dead codes in PartitionTestUtils --- .../runtime/io/network/partition/PartitionTestUtils.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java index b1a58a0053..42b0b1a838 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java @@ -112,10 +112,6 @@ public enum PartitionTestUtils { true); } - public static ResultPartitionDeploymentDescriptor createPartitionDeploymentDescriptor() { - return createPartitionDeploymentDescriptor(ResultPartitionType.BLOCKING); - } - public static void writeBuffers( ResultPartitionWriter partition, int numberOfBuffers, -- Gitee From ca206d8d620b3a0c6c7bce884e656433b6e2dfb2 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Mon, 23 Sep 2019 14:42:46 +0800 Subject: [PATCH 064/268] [hotfix][tests] Adjust to use constant BUFFER_SIZE in PipelinedSubpartitionWithReadViewTest --- .../partition/PipelinedSubpartitionWithReadViewTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java index e945342f99..56945016ca 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java @@ -308,7 +308,7 @@ public class PipelinedSubpartitionWithReadViewTest { final int numberOfAddedBuffers = 5; for (int i = 1; i <= numberOfAddedBuffers; i++) { - final BufferBuilder bufferBuilder = createFilledBufferBuilder(1024, 10); + final BufferBuilder bufferBuilder = createFilledBufferBuilder(BUFFER_SIZE); subpartition.add(bufferBuilder.createBufferConsumer()); if (i < numberOfAddedBuffers || isFinished) { -- Gitee From 7f2382a532ab72e6ead12945eb2395e6af829eb5 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Mon, 23 Sep 2019 16:04:30 +0800 Subject: [PATCH 065/268] [FLINK-10995][runtime] Copy intermediate serialization results only once for broadcast mode The behavior of current channel selector is either for one channel or all the channels for broadcast mode. In broadcast mode, the intermediate serialization results would be copied into every BufferBuilder requested for every sub partition, so this would affect the performance seriously especially in large scale jobs. We can copy to only one target BufferBuilder and the corresponding BufferConsumer would be shared by all the sub partitions to improve the performance. For mixed operations with broadcast and non-broadcast, we should finish the previous BufferBuilder first before transforming from broadcast to non-broadcast, vice versa. --- .../api/writer/BroadcastRecordWriter.java | 144 ++++++++++++- .../writer/ChannelSelectorRecordWriter.java | 142 +++++++++++++ .../io/network/api/writer/RecordWriter.java | 145 +++++-------- .../api/writer/RecordWriterBuilder.java | 4 +- .../io/network/buffer/BufferBuilder.java | 13 +- .../io/network/buffer/BufferConsumer.java | 11 +- .../api/writer/BroadcastRecordWriterTest.java | 112 ++++++++++ .../network/api/writer/RecordWriterTest.java | 199 ++++++++++-------- .../buffer/BufferBuilderAndConsumerTest.java | 19 +- .../buffer/BufferBuilderTestUtils.java | 28 ++- ...dBlockingSubpartitionAvailabilityTest.java | 2 +- .../partition/FileChannelBoundedDataTest.java | 4 +- .../partition/InputGateFairnessTest.java | 6 +- .../network/partition/PartitionTestUtils.java | 4 +- .../partition/PipelinedSubpartitionTest.java | 11 +- ...PipelinedSubpartitionWithReadViewTest.java | 61 +++--- .../partition/ResultPartitionTest.java | 12 +- .../partition/SubpartitionTestBase.java | 14 +- .../IteratorWrappingTestSingleInputGate.java | 5 +- .../consumer/LocalInputChannelTest.java | 4 +- .../consumer/SingleInputGateTest.java | 2 +- .../consumer/StreamTestSingleInputGate.java | 5 +- .../io/StreamTaskNetworkInputTest.java | 7 +- 23 files changed, 674 insertions(+), 280 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriterTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java index effff59a10..7544af2017 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java @@ -18,30 +18,164 @@ package org.apache.flink.runtime.io.network.api.writer; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import java.io.IOException; +import java.util.Optional; + +import static org.apache.flink.util.Preconditions.checkState; /** * A special record-oriented runtime result writer only for broadcast mode. * - *

The BroadcastRecordWriter extends the {@link RecordWriter} and handles {@link #emit(IOReadableWritable)} - * operation via {@link #broadcastEmit(IOReadableWritable)} directly in a more efficient way. + *

The BroadcastRecordWriter extends the {@link RecordWriter} and maintain a single {@link BufferBuilder} + * for all the channels. Then the serialization results need be copied only once to this buffer which would be + * shared for all the channels in a more efficient way. * * @param the type of the record that can be emitted with this record writer */ -public class BroadcastRecordWriter extends RecordWriter { +public final class BroadcastRecordWriter extends RecordWriter { + + /** The current buffer builder shared for all the channels. */ + private Optional bufferBuilder = Optional.empty(); + + /** + * The flag for judging whether {@link #requestNewBufferBuilder(int)} and {@link #flushTargetPartition(int)} + * is triggered by {@link #randomEmit(IOReadableWritable)} or not. + */ + private boolean randomTriggered; BroadcastRecordWriter( ResultPartitionWriter writer, - ChannelSelector channelSelector, long timeout, String taskName) { - super(writer, channelSelector, timeout, taskName); + super(writer, timeout, taskName); } @Override public void emit(T record) throws IOException, InterruptedException { broadcastEmit(record); } + + @Override + public void randomEmit(T record) throws IOException, InterruptedException { + randomEmit(record, rng.nextInt(numberOfChannels)); + } + + /** + * For non-broadcast emit, we try to finish the current {@link BufferBuilder} first, and then request + * a new {@link BufferBuilder} for the random channel. If this new {@link BufferBuilder} is not full, + * it can be shared for all the other channels via initializing readable position in created + * {@link BufferConsumer}. + */ + @VisibleForTesting + void randomEmit(T record, int targetChannelIndex) throws IOException, InterruptedException { + tryFinishCurrentBufferBuilder(targetChannelIndex); + + randomTriggered = true; + emit(record, targetChannelIndex); + randomTriggered = false; + + if (bufferBuilder.isPresent()) { + for (int index = 0; index < numberOfChannels; index++) { + if (index != targetChannelIndex) { + targetPartition.addBufferConsumer(bufferBuilder.get().createBufferConsumer(), index); + } + } + } + } + + @Override + public void broadcastEmit(T record) throws IOException, InterruptedException { + // We could actually select any target channel here because all the channels + // are sharing the same BufferBuilder in broadcast mode. + emit(record, 0); + } + + /** + * The flush could be triggered by {@link #randomEmit(IOReadableWritable)}, {@link #emit(IOReadableWritable)} + * or {@link #broadcastEmit(IOReadableWritable)}. Only random emit should flush a single target channel, + * otherwise we should flush all the channels. + */ + @Override + public void flushTargetPartition(int targetChannel) { + if (randomTriggered) { + super.flushTargetPartition(targetChannel); + } else { + flushAll(); + } + } + + @Override + public BufferBuilder getBufferBuilder(int targetChannel) throws IOException, InterruptedException { + if (bufferBuilder.isPresent()) { + return bufferBuilder.get(); + } else { + return requestNewBufferBuilder(targetChannel); + } + } + + /** + * The request could be from broadcast or non-broadcast modes like {@link #randomEmit(IOReadableWritable)}. + * + *

For non-broadcast, the created {@link BufferConsumer} is only for the target channel. + * + *

For broadcast, all the channels share the same requested {@link BufferBuilder} and the created + * {@link BufferConsumer} is copied for every channel. + */ + @Override + public BufferBuilder requestNewBufferBuilder(int targetChannel) throws IOException, InterruptedException { + checkState(!bufferBuilder.isPresent() || bufferBuilder.get().isFinished()); + + BufferBuilder builder = targetPartition.getBufferBuilder(); + if (randomTriggered) { + targetPartition.addBufferConsumer(builder.createBufferConsumer(), targetChannel); + } else { + try (BufferConsumer bufferConsumer = builder.createBufferConsumer()) { + for (int channel = 0; channel < numberOfChannels; channel++) { + targetPartition.addBufferConsumer(bufferConsumer.copy(), channel); + } + } + } + + bufferBuilder = Optional.of(builder); + return builder; + } + + @Override + public void tryFinishCurrentBufferBuilder(int targetChannel) { + if (!bufferBuilder.isPresent()) { + return; + } + + BufferBuilder builder = bufferBuilder.get(); + bufferBuilder = Optional.empty(); + + finishBufferBuilder(builder); + } + + @Override + public void emptyCurrentBufferBuilder(int targetChannel) { + bufferBuilder = Optional.empty(); + } + + @Override + public void closeBufferBuilder(int targetChannel) { + closeBufferBuilder(); + } + + @Override + public void clearBuffers() { + closeBufferBuilder(); + } + + private void closeBufferBuilder() { + if (bufferBuilder.isPresent()) { + bufferBuilder.get().finish(); + bufferBuilder = Optional.empty(); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java new file mode 100644 index 0000000000..eaeb5091f6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.api.writer; + +import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; + +import java.io.IOException; +import java.util.Optional; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * A regular record-oriented runtime result writer. + * + *

The ChannelSelectorRecordWriter extends the {@link RecordWriter} and maintains an array of + * {@link BufferBuilder}s for all the channels. The {@link #emit(IOReadableWritable)} + * operation is based on {@link ChannelSelector} to select the target channel. + * + * @param the type of the record that can be emitted with this record writer + */ +public final class ChannelSelectorRecordWriter extends RecordWriter { + + private final ChannelSelector channelSelector; + + private final Optional[] bufferBuilders; + + ChannelSelectorRecordWriter( + ResultPartitionWriter writer, + ChannelSelector channelSelector, + long timeout, + String taskName) { + super(writer, timeout, taskName); + + this.channelSelector = checkNotNull(channelSelector); + this.channelSelector.setup(numberOfChannels); + + this.bufferBuilders = new Optional[numberOfChannels]; + for (int i = 0; i < numberOfChannels; i++) { + bufferBuilders[i] = Optional.empty(); + } + } + + @Override + public void emit(T record) throws IOException, InterruptedException { + emit(record, channelSelector.selectChannel(record)); + } + + @Override + public void randomEmit(T record) throws IOException, InterruptedException { + emit(record, rng.nextInt(numberOfChannels)); + } + + /** + * The record is serialized into intermediate serialization buffer which is then copied + * into the target buffer for every channel. + */ + @Override + public void broadcastEmit(T record) throws IOException, InterruptedException { + checkErroneous(); + + serializer.serializeRecord(record); + + boolean pruneAfterCopying = false; + for (int targetChannel = 0; targetChannel < numberOfChannels; targetChannel++) { + if (copyFromSerializerToTargetChannel(targetChannel)) { + pruneAfterCopying = true; + } + } + + if (pruneAfterCopying) { + serializer.prune(); + } + } + + @Override + public BufferBuilder getBufferBuilder(int targetChannel) throws IOException, InterruptedException { + if (bufferBuilders[targetChannel].isPresent()) { + return bufferBuilders[targetChannel].get(); + } else { + return requestNewBufferBuilder(targetChannel); + } + } + + @Override + public BufferBuilder requestNewBufferBuilder(int targetChannel) throws IOException, InterruptedException { + checkState(!bufferBuilders[targetChannel].isPresent() || bufferBuilders[targetChannel].get().isFinished()); + + BufferBuilder bufferBuilder = targetPartition.getBufferBuilder(); + targetPartition.addBufferConsumer(bufferBuilder.createBufferConsumer(), targetChannel); + bufferBuilders[targetChannel] = Optional.of(bufferBuilder); + return bufferBuilder; + } + + @Override + public void tryFinishCurrentBufferBuilder(int targetChannel) { + if (!bufferBuilders[targetChannel].isPresent()) { + return; + } + BufferBuilder bufferBuilder = bufferBuilders[targetChannel].get(); + bufferBuilders[targetChannel] = Optional.empty(); + + finishBufferBuilder(bufferBuilder); + } + + @Override + public void emptyCurrentBufferBuilder(int targetChannel) { + bufferBuilders[targetChannel] = Optional.empty(); + } + + @Override + public void closeBufferBuilder(int targetChannel) { + if (bufferBuilders[targetChannel].isPresent()) { + bufferBuilders[targetChannel].get().finish(); + bufferBuilders[targetChannel] = Optional.empty(); + } + } + + @Override + public void clearBuffers() { + for (int index = 0; index < numberOfChannels; index++) { + closeBufferBuilder(index); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java index cc40df064e..b648eef345 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java @@ -42,7 +42,7 @@ import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkState; /** - * A record-oriented runtime result writer. + * An abstract record-oriented runtime result writer. * *

The RecordWriter wraps the runtime's {@link ResultPartitionWriter} and takes care of * serializing records into buffers. @@ -54,23 +54,17 @@ import static org.apache.flink.util.Preconditions.checkState; * * @param the type of the record that can be emitted with this record writer */ -public class RecordWriter { +public abstract class RecordWriter { private static final Logger LOG = LoggerFactory.getLogger(RecordWriter.class); - private final ResultPartitionWriter targetPartition; + protected final ResultPartitionWriter targetPartition; - private final ChannelSelector channelSelector; + protected final int numberOfChannels; - private final int numberOfChannels; + protected final RecordSerializer serializer; - private final int[] broadcastChannels; - - private final RecordSerializer serializer; - - private final Optional[] bufferBuilders; - - private final Random rng = new XORShiftRandom(); + protected final Random rng = new XORShiftRandom(); private Counter numBytesOut = new SimpleCounter(); @@ -87,19 +81,11 @@ public class RecordWriter { /** To avoid synchronization overhead on the critical path, best-effort error tracking is enough here.*/ private Throwable flusherException; - RecordWriter(ResultPartitionWriter writer, ChannelSelector channelSelector, long timeout, String taskName) { + RecordWriter(ResultPartitionWriter writer, long timeout, String taskName) { this.targetPartition = writer; - this.channelSelector = channelSelector; this.numberOfChannels = writer.getNumberOfSubpartitions(); - this.channelSelector.setup(numberOfChannels); this.serializer = new SpanningRecordSerializer(); - this.bufferBuilders = new Optional[numberOfChannels]; - this.broadcastChannels = new int[numberOfChannels]; - for (int i = 0; i < numberOfChannels; i++) { - broadcastChannels[i] = i; - bufferBuilders[i] = Optional.empty(); - } checkArgument(timeout >= -1); this.flushAlways = (timeout == 0); @@ -115,42 +101,12 @@ public class RecordWriter { } } - public void emit(T record) throws IOException, InterruptedException { + protected void emit(T record, int targetChannel) throws IOException, InterruptedException { checkErroneous(); - emit(record, channelSelector.selectChannel(record)); - } - /** - * This is used to broadcast Streaming Watermarks in-band with records. This ignores - * the {@link ChannelSelector}. - */ - public void broadcastEmit(T record) throws IOException, InterruptedException { - checkErroneous(); serializer.serializeRecord(record); - boolean pruneAfterCopying = false; - for (int channel : broadcastChannels) { - if (copyFromSerializerToTargetChannel(channel)) { - pruneAfterCopying = true; - } - } - // Make sure we don't hold onto the large intermediate serialization buffer for too long - if (pruneAfterCopying) { - serializer.prune(); - } - } - - /** - * This is used to send LatencyMarks to a random target channel. - */ - public void randomEmit(T record) throws IOException, InterruptedException { - emit(record, rng.nextInt(numberOfChannels)); - } - - private void emit(T record, int targetChannel) throws IOException, InterruptedException { - serializer.serializeRecord(record); - if (copyFromSerializerToTargetChannel(targetChannel)) { serializer.prune(); } @@ -160,7 +116,7 @@ public class RecordWriter { * @param targetChannel * @return true if the intermediate serialization buffer should be pruned */ - private boolean copyFromSerializerToTargetChannel(int targetChannel) throws IOException, InterruptedException { + protected boolean copyFromSerializerToTargetChannel(int targetChannel) throws IOException, InterruptedException { // We should reset the initial position of the intermediate serialization buffer before // copying, so the serialization results can be copied to multiple target buffers. serializer.reset(); @@ -169,15 +125,14 @@ public class RecordWriter { BufferBuilder bufferBuilder = getBufferBuilder(targetChannel); SerializationResult result = serializer.copyToBufferBuilder(bufferBuilder); while (result.isFullBuffer()) { - numBytesOut.inc(bufferBuilder.finish()); - numBuffersOut.inc(); + finishBufferBuilder(bufferBuilder); // If this was a full record, we are done. Not breaking out of the loop at this point // will lead to another buffer request before breaking out (that would not be a // problem per se, but it can lead to stalls in the pipeline). if (result.isFullRecord()) { pruneTriggered = true; - bufferBuilders[targetChannel] = Optional.empty(); + emptyCurrentBufferBuilder(targetChannel); break; } @@ -187,7 +142,7 @@ public class RecordWriter { checkState(!serializer.hasSerializedData(), "All data should be written at once"); if (flushAlways) { - targetPartition.flush(targetChannel); + flushTargetPartition(targetChannel); } return pruneTriggered; } @@ -211,10 +166,8 @@ public class RecordWriter { targetPartition.flushAll(); } - public void clearBuffers() { - for (int targetChannel = 0; targetChannel < numberOfChannels; targetChannel++) { - closeBufferBuilder(targetChannel); - } + protected void flushTargetPartition(int targetChannel) { + targetPartition.flush(targetChannel); } /** @@ -225,46 +178,56 @@ public class RecordWriter { numBuffersOut = metrics.getNumBuffersOutCounter(); } - /** - * Marks the current {@link BufferBuilder} as finished and clears the state for next one. - */ - private void tryFinishCurrentBufferBuilder(int targetChannel) { - if (!bufferBuilders[targetChannel].isPresent()) { - return; - } - BufferBuilder bufferBuilder = bufferBuilders[targetChannel].get(); - bufferBuilders[targetChannel] = Optional.empty(); + protected void finishBufferBuilder(BufferBuilder bufferBuilder) { numBytesOut.inc(bufferBuilder.finish()); numBuffersOut.inc(); } + /** + * This is used to send regular records. + */ + public abstract void emit(T record) throws IOException, InterruptedException; + + /** + * This is used to send LatencyMarks to a random target channel. + */ + public abstract void randomEmit(T record) throws IOException, InterruptedException; + + /** + * This is used to broadcast streaming Watermarks in-band with records. + */ + public abstract void broadcastEmit(T record) throws IOException, InterruptedException; + /** * The {@link BufferBuilder} may already exist if not filled up last time, otherwise we need * request a new one for this target channel. */ - private BufferBuilder getBufferBuilder(int targetChannel) throws IOException, InterruptedException { - if (bufferBuilders[targetChannel].isPresent()) { - return bufferBuilders[targetChannel].get(); - } else { - return requestNewBufferBuilder(targetChannel); - } - } + abstract BufferBuilder getBufferBuilder(int targetChannel) throws IOException, InterruptedException; - private BufferBuilder requestNewBufferBuilder(int targetChannel) throws IOException, InterruptedException { - checkState(!bufferBuilders[targetChannel].isPresent() || bufferBuilders[targetChannel].get().isFinished()); + /** + * Requests a new {@link BufferBuilder} for the target channel and returns it. + */ + abstract BufferBuilder requestNewBufferBuilder(int targetChannel) throws IOException, InterruptedException; - BufferBuilder bufferBuilder = targetPartition.getBufferBuilder(); - bufferBuilders[targetChannel] = Optional.of(bufferBuilder); - targetPartition.addBufferConsumer(bufferBuilder.createBufferConsumer(), targetChannel); - return bufferBuilder; - } + /** + * Marks the current {@link BufferBuilder} as finished if present and clears the state for next one. + */ + abstract void tryFinishCurrentBufferBuilder(int targetChannel); - private void closeBufferBuilder(int targetChannel) { - if (bufferBuilders[targetChannel].isPresent()) { - bufferBuilders[targetChannel].get().finish(); - bufferBuilders[targetChannel] = Optional.empty(); - } - } + /** + * Marks the current {@link BufferBuilder} as empty for the target channel. + */ + abstract void emptyCurrentBufferBuilder(int targetChannel); + + /** + * Marks the current {@link BufferBuilder} as finished and releases the resources for the target channel. + */ + abstract void closeBufferBuilder(int targetChannel); + + /** + * Closes the {@link BufferBuilder}s for all the channels. + */ + public abstract void clearBuffers(); /** * Closes the writer. This stops the flushing thread (if there is one). @@ -296,7 +259,7 @@ public class RecordWriter { } } - private void checkErroneous() throws IOException { + protected void checkErroneous() throws IOException { if (flusherException != null) { throw new IOException("An exception happened while flushing the outputs", flusherException); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterBuilder.java index 79b372b6bc..365ca2096e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterBuilder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterBuilder.java @@ -46,9 +46,9 @@ public class RecordWriterBuilder { public RecordWriter build(ResultPartitionWriter writer) { if (selector.isBroadcast()) { - return new BroadcastRecordWriter(writer, selector, timeout, taskName); + return new BroadcastRecordWriter(writer, timeout, taskName); } else { - return new RecordWriter(writer, selector, timeout, taskName); + return new ChannelSelectorRecordWriter(writer, selector, timeout, taskName); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferBuilder.java index 6fb067ef8c..bcd42d23b1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferBuilder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferBuilder.java @@ -40,24 +40,23 @@ public class BufferBuilder { private final SettablePositionMarker positionMarker = new SettablePositionMarker(); - private boolean bufferConsumerCreated = false; - public BufferBuilder(MemorySegment memorySegment, BufferRecycler recycler) { this.memorySegment = checkNotNull(memorySegment); this.recycler = checkNotNull(recycler); } /** - * @return created matching instance of {@link BufferConsumer} to this {@link BufferBuilder}. There can exist only - * one {@link BufferConsumer} per each {@link BufferBuilder} and vice versa. + * This method always creates a {@link BufferConsumer} starting from the current writer offset. Data written to + * {@link BufferBuilder} before creation of {@link BufferConsumer} won't be visible for that {@link BufferConsumer}. + * + * @return created matching instance of {@link BufferConsumer} to this {@link BufferBuilder}. */ public BufferConsumer createBufferConsumer() { - checkState(!bufferConsumerCreated, "There can not exists two BufferConsumer for one BufferBuilder"); - bufferConsumerCreated = true; return new BufferConsumer( memorySegment, recycler, - positionMarker); + positionMarker, + positionMarker.cachedPosition); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java index b58a627dfe..c06c233746 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java @@ -45,16 +45,17 @@ public class BufferConsumer implements Closeable { private int currentReaderPosition; /** - * Constructs {@link BufferConsumer} instance with content that can be changed by {@link BufferBuilder}. + * Constructs {@link BufferConsumer} instance with the initial reader position. */ public BufferConsumer( MemorySegment memorySegment, BufferRecycler recycler, - PositionMarker currentWriterPosition) { + PositionMarker currentWriterPosition, + int currentReaderPosition) { this( new NetworkBuffer(checkNotNull(memorySegment), checkNotNull(recycler), true), currentWriterPosition, - 0); + currentReaderPosition); } /** @@ -136,6 +137,10 @@ public class BufferConsumer implements Closeable { return writerPosition.getCached(); } + int getCurrentReaderPosition() { + return currentReaderPosition; + } + /** * Cached reading wrapper around {@link PositionMarker}. * diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriterTest.java new file mode 100644 index 0000000000..d95b87c836 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriterTest.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.api.writer; + +import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer; +import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; +import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider; +import org.apache.flink.testutils.serialization.types.SerializationTestType; +import org.apache.flink.testutils.serialization.types.SerializationTestTypeFactory; +import org.apache.flink.testutils.serialization.types.Util; + +import org.junit.Test; + +import java.util.ArrayDeque; +import java.util.HashMap; +import java.util.Map; +import java.util.Queue; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for the {@link BroadcastRecordWriter}. + */ +public class BroadcastRecordWriterTest extends RecordWriterTest { + + public BroadcastRecordWriterTest() { + super(true); + } + + /** + * Tests the number of requested buffers and results are correct in the case of switching + * modes between {@link BroadcastRecordWriter#broadcastEmit(IOReadableWritable)} and + * {@link BroadcastRecordWriter#randomEmit(IOReadableWritable)}. + */ + @Test + public void testBroadcastMixedRandomEmitRecord() throws Exception { + final int numberOfChannels = 4; + final int numberOfRecords = 8; + final int bufferSize = 32; + + @SuppressWarnings("unchecked") + final Queue[] queues = new Queue[numberOfChannels]; + for (int i = 0; i < numberOfChannels; i++) { + queues[i] = new ArrayDeque<>(); + } + + final TestPooledBufferProvider bufferProvider = new TestPooledBufferProvider(Integer.MAX_VALUE, bufferSize); + final ResultPartitionWriter partitionWriter = new CollectingPartitionWriter(queues, bufferProvider); + final BroadcastRecordWriter writer = new BroadcastRecordWriter<>(partitionWriter, 0, "test"); + final RecordDeserializer deserializer = new SpillingAdaptiveSpanningRecordDeserializer<>( + new String[]{ tempFolder.getRoot().getAbsolutePath() }); + + // generate the configured number of int values as global record set + final Iterable records = Util.randomRecords(numberOfRecords, SerializationTestTypeFactory.INT); + // restore the corresponding record set for every input channel + final Map> serializedRecords = new HashMap<>(); + for (int i = 0; i < numberOfChannels; i++) { + serializedRecords.put(i, new ArrayDeque<>()); + } + + // every record in global set would both emit into one random channel and broadcast to all the channels + int index = 0; + for (SerializationTestType record : records) { + int randomChannel = index++ % numberOfChannels; + writer.randomEmit(record, randomChannel); + serializedRecords.get(randomChannel).add(record); + + writer.broadcastEmit(record); + for (int i = 0; i < numberOfChannels; i++) { + serializedRecords.get(i).add(record); + } + } + + final int numberOfCreatedBuffers = bufferProvider.getNumberOfCreatedBuffers(); + // verify the expected number of requested buffers, and it would always request a new buffer while random emitting + assertEquals(numberOfRecords, numberOfCreatedBuffers); + + for (int i = 0; i < numberOfChannels; i++) { + // every channel would queue the number of above crated buffers + assertEquals(numberOfRecords, queues[i].size()); + + final int excessRandomRecords = i < numberOfRecords % numberOfChannels ? 1 : 0; + final int numberOfRandomRecords = numberOfRecords / numberOfChannels + excessRandomRecords; + final int numberOfTotalRecords = numberOfRecords + numberOfRandomRecords; + // verify the data correctness in every channel queue + verifyDeserializationResults( + queues[i], + deserializer, + serializedRecords.get(i), + numberOfCreatedBuffers, + numberOfTotalRecords); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java index 882f83c5a9..239c47d3a5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java @@ -83,6 +83,16 @@ import static org.mockito.Mockito.when; */ public class RecordWriterTest { + private final boolean isBroadcastWriter; + + public RecordWriterTest() { + this(false); + } + + RecordWriterTest(boolean isBroadcastWriter) { + this.isBroadcastWriter = isBroadcastWriter; + } + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); @@ -132,7 +142,7 @@ public class RecordWriterTest { ResultPartitionWriter partitionWriter = new RecyclingPartitionWriter(bufferProvider); - final RecordWriter recordWriter = new RecordWriterBuilder().build(partitionWriter); + final RecordWriter recordWriter = createRecordWriter(partitionWriter); Future result = executor.submit(new Callable() { @Override @@ -184,7 +194,7 @@ public class RecordWriterTest { ResultPartitionWriter partitionWriter = spy(new RecyclingPartitionWriter(new TestPooledBufferProvider(1, 16))); - RecordWriter recordWriter = new RecordWriterBuilder().build(partitionWriter); + RecordWriter recordWriter = createRecordWriter(partitionWriter); // Fill a buffer, but don't write it out. recordWriter.emit(new IntValue(0)); @@ -214,7 +224,7 @@ public class RecordWriterTest { TestPooledBufferProvider bufferProvider = new TestPooledBufferProvider(Integer.MAX_VALUE, bufferSize); ResultPartitionWriter partitionWriter = new CollectingPartitionWriter(queues, bufferProvider); - RecordWriter writer = new RecordWriterBuilder().build(partitionWriter); + RecordWriter writer = createRecordWriter(partitionWriter); CheckpointBarrier barrier = new CheckpointBarrier(Integer.MAX_VALUE + 919192L, Integer.MAX_VALUE + 18828228L, CheckpointOptions.forCheckpointWithDefaultLocation()); // No records emitted yet, broadcast should not request a buffer @@ -251,53 +261,66 @@ public class RecordWriterTest { TestPooledBufferProvider bufferProvider = new TestPooledBufferProvider(Integer.MAX_VALUE, bufferSize); ResultPartitionWriter partitionWriter = new CollectingPartitionWriter(queues, bufferProvider); - RecordWriter writer = new RecordWriterBuilder().build(partitionWriter); + RecordWriter writer = createRecordWriter(partitionWriter); CheckpointBarrier barrier = new CheckpointBarrier(Integer.MAX_VALUE + 1292L, Integer.MAX_VALUE + 199L, CheckpointOptions.forCheckpointWithDefaultLocation()); // Emit records on some channels first (requesting buffers), then // broadcast the event. The record buffers should be emitted first, then // the event. After the event, no new buffer should be requested. - // (i) Smaller than the buffer size (single buffer request => 1) + // (i) Smaller than the buffer size byte[] bytes = new byte[bufferSize / 2]; rand.nextBytes(bytes); writer.emit(new ByteArrayIO(bytes)); - // (ii) Larger than the buffer size (two buffer requests => 1 + 2) + // (ii) Larger than the buffer size bytes = new byte[bufferSize + 1]; rand.nextBytes(bytes); writer.emit(new ByteArrayIO(bytes)); - // (iii) Exactly the buffer size (single buffer request => 1 + 2 + 1) + // (iii) Exactly the buffer size bytes = new byte[bufferSize - lenBytes]; rand.nextBytes(bytes); writer.emit(new ByteArrayIO(bytes)); - // (iv) Nothing on the 4th channel (no buffer request => 1 + 2 + 1 + 0 = 4) - - // (v) Broadcast the event + // (iv) Broadcast the event writer.broadcastEvent(barrier); - assertEquals(4, bufferProvider.getNumberOfCreatedBuffers()); + if (isBroadcastWriter) { + assertEquals(3, bufferProvider.getNumberOfCreatedBuffers()); - BufferOrEvent boe; - assertEquals(2, queues[0].size()); // 1 buffer + 1 event - assertTrue(parseBuffer(queues[0].remove(), 0).isBuffer()); - assertEquals(3, queues[1].size()); // 2 buffers + 1 event - assertTrue(parseBuffer(queues[1].remove(), 1).isBuffer()); - assertTrue(parseBuffer(queues[1].remove(), 1).isBuffer()); - assertEquals(2, queues[2].size()); // 1 buffer + 1 event - assertTrue(parseBuffer(queues[2].remove(), 2).isBuffer()); - assertEquals(1, queues[3].size()); // 0 buffers + 1 event + for (int i = 0; i < numberOfChannels; i++) { + assertEquals(4, queues[i].size()); // 3 buffer + 1 event - // every queue's last element should be the event - for (int i = 0; i < numberOfChannels; i++) { - boe = parseBuffer(queues[i].remove(), i); - assertTrue(boe.isEvent()); - assertEquals(barrier, boe.getEvent()); + for (int j = 0; j < 3; j++) { + assertTrue(parseBuffer(queues[i].remove(), 0).isBuffer()); + } + + BufferOrEvent boe = parseBuffer(queues[i].remove(), i); + assertTrue(boe.isEvent()); + assertEquals(barrier, boe.getEvent()); + } + } else { + assertEquals(4, bufferProvider.getNumberOfCreatedBuffers()); + + assertEquals(2, queues[0].size()); // 1 buffer + 1 event + assertTrue(parseBuffer(queues[0].remove(), 0).isBuffer()); + assertEquals(3, queues[1].size()); // 2 buffers + 1 event + assertTrue(parseBuffer(queues[1].remove(), 1).isBuffer()); + assertTrue(parseBuffer(queues[1].remove(), 1).isBuffer()); + assertEquals(2, queues[2].size()); // 1 buffer + 1 event + assertTrue(parseBuffer(queues[2].remove(), 2).isBuffer()); + assertEquals(1, queues[3].size()); // 0 buffers + 1 event + + // every queue's last element should be the event + for (int i = 0; i < numberOfChannels; i++) { + BufferOrEvent boe = parseBuffer(queues[i].remove(), i); + assertTrue(boe.isEvent()); + assertEquals(barrier, boe.getEvent()); + } } } @@ -313,7 +336,7 @@ public class RecordWriterTest { ResultPartitionWriter partition = new CollectingPartitionWriter(queues, new TestPooledBufferProvider(Integer.MAX_VALUE)); - RecordWriter writer = new RecordWriterBuilder().build(partition); + RecordWriter writer = createRecordWriter(partition); writer.broadcastEvent(EndOfPartitionEvent.INSTANCE); @@ -352,21 +375,46 @@ public class RecordWriterTest { verifyBroadcastBufferOrEventIndependence(false); } - /** - * Tests that records are broadcast via {@link ChannelSelector} and - * {@link RecordWriter#emit(IOReadableWritable)}. - */ - @Test - public void testEmitRecordWithBroadcastPartitioner() throws Exception { - emitRecordWithBroadcastPartitionerOrBroadcastEmitRecord(false); - } - /** * Tests that records are broadcast via {@link RecordWriter#broadcastEmit(IOReadableWritable)}. */ @Test public void testBroadcastEmitRecord() throws Exception { - emitRecordWithBroadcastPartitionerOrBroadcastEmitRecord(true); + final int numberOfChannels = 4; + final int bufferSize = 32; + final int numValues = 8; + final int serializationLength = 4; + + @SuppressWarnings("unchecked") + final Queue[] queues = new Queue[numberOfChannels]; + for (int i = 0; i < numberOfChannels; i++) { + queues[i] = new ArrayDeque<>(); + } + + final TestPooledBufferProvider bufferProvider = new TestPooledBufferProvider(Integer.MAX_VALUE, bufferSize); + final ResultPartitionWriter partitionWriter = new CollectingPartitionWriter(queues, bufferProvider); + final RecordWriter writer = createRecordWriter(partitionWriter); + final RecordDeserializer deserializer = new SpillingAdaptiveSpanningRecordDeserializer<>( + new String[]{ tempFolder.getRoot().getAbsolutePath() }); + + final ArrayDeque serializedRecords = new ArrayDeque<>(); + final Iterable records = Util.randomRecords(numValues, SerializationTestTypeFactory.INT); + for (SerializationTestType record : records) { + serializedRecords.add(record); + writer.broadcastEmit(record); + } + + final int numRequiredBuffers = numValues / (bufferSize / (4 + serializationLength)); + if (isBroadcastWriter) { + assertEquals(numRequiredBuffers, bufferProvider.getNumberOfCreatedBuffers()); + } else { + assertEquals(numRequiredBuffers * numberOfChannels, bufferProvider.getNumberOfCreatedBuffers()); + } + + for (int i = 0; i < numberOfChannels; i++) { + assertEquals(numRequiredBuffers, queues[i].size()); + verifyDeserializationResults(queues[i], deserializer, serializedRecords.clone(), numRequiredBuffers, numValues); + } } private void verifyBroadcastBufferOrEventIndependence(boolean broadcastEvent) throws Exception { @@ -375,7 +423,7 @@ public class RecordWriterTest { ResultPartitionWriter partition = new CollectingPartitionWriter(queues, new TestPooledBufferProvider(Integer.MAX_VALUE)); - RecordWriter writer = new RecordWriterBuilder().build(partition); + RecordWriter writer = createRecordWriter(partition); if (broadcastEvent) { writer.broadcastEvent(EndOfPartitionEvent.INSTANCE); @@ -396,59 +444,32 @@ public class RecordWriterTest { assertEquals("Buffer 2 shares the same reader index as buffer 1", 0, buffer2.getReaderIndex()); } - /** - * The results of emitting records via BroadcastPartitioner or broadcasting records directly are the same, - * that is all the target channels can receive the whole outputs. - * - * @param isBroadcastEmit whether using {@link RecordWriter#broadcastEmit(IOReadableWritable)} or not - */ - private void emitRecordWithBroadcastPartitionerOrBroadcastEmitRecord(boolean isBroadcastEmit) throws Exception { - final int numberOfChannels = 4; - final int bufferSize = 32; - final int numValues = 8; - final int serializationLength = 4; + protected void verifyDeserializationResults( + Queue queue, + RecordDeserializer deserializer, + ArrayDeque expectedRecords, + int numRequiredBuffers, + int numValues) throws Exception { + int assertRecords = 0; + for (int j = 0; j < numRequiredBuffers; j++) { + Buffer buffer = buildSingleBuffer(queue.remove()); + deserializer.setNextBuffer(buffer); - @SuppressWarnings("unchecked") - final Queue[] queues = new Queue[numberOfChannels]; - for (int i = 0; i < numberOfChannels; i++) { - queues[i] = new ArrayDeque<>(); + assertRecords += DeserializationUtils.deserializeRecords(expectedRecords, deserializer); } + Assert.assertEquals(numValues, assertRecords); + } - final TestPooledBufferProvider bufferProvider = new TestPooledBufferProvider(Integer.MAX_VALUE, bufferSize); - final ResultPartitionWriter partitionWriter = new CollectingPartitionWriter(queues, bufferProvider); - final ChannelSelector selector = new OutputEmitter(ShipStrategyType.BROADCAST, 0); - final RecordWriter writer = new RecordWriterBuilder() - .setChannelSelector(selector) - .setTimeout(0) - .build(partitionWriter); - final RecordDeserializer deserializer = new SpillingAdaptiveSpanningRecordDeserializer<>( - new String[]{ tempFolder.getRoot().getAbsolutePath() }); - - final ArrayDeque serializedRecords = new ArrayDeque<>(); - final Iterable records = Util.randomRecords(numValues, SerializationTestTypeFactory.INT); - for (SerializationTestType record : records) { - serializedRecords.add(record); - - if (isBroadcastEmit) { - writer.broadcastEmit(record); - } else { - writer.emit(record); - } - } - - final int requiredBuffers = numValues / (bufferSize / (4 + serializationLength)); - for (int i = 0; i < numberOfChannels; i++) { - assertEquals(requiredBuffers, queues[i].size()); - - final ArrayDeque expectedRecords = serializedRecords.clone(); - int assertRecords = 0; - for (int j = 0; j < requiredBuffers; j++) { - Buffer buffer = buildSingleBuffer(queues[i].remove()); - deserializer.setNextBuffer(buffer); - - assertRecords += DeserializationUtils.deserializeRecords(expectedRecords, deserializer); - } - Assert.assertEquals(numValues, assertRecords); + /** + * Creates the {@link RecordWriter} instance based on whether it is a broadcast writer. + */ + private RecordWriter createRecordWriter(ResultPartitionWriter writer) { + if (isBroadcastWriter) { + return new RecordWriterBuilder() + .setChannelSelector(new OutputEmitter(ShipStrategyType.BROADCAST, 0)) + .build(writer); + } else { + return new RecordWriterBuilder().build(writer); } } @@ -459,7 +480,7 @@ public class RecordWriterTest { /** * Partition writer that collects the added buffers/events in multiple queue. */ - private static class CollectingPartitionWriter implements ResultPartitionWriter { + static class CollectingPartitionWriter implements ResultPartitionWriter { private final Queue[] queues; private final BufferProvider bufferProvider; private final ResultPartitionID partitionId = new ResultPartitionID(); @@ -470,7 +491,7 @@ public class RecordWriterTest { * @param queues one queue per outgoing channel * @param bufferProvider buffer provider */ - private CollectingPartitionWriter(Queue[] queues, BufferProvider bufferProvider) { + CollectingPartitionWriter(Queue[] queues, BufferProvider bufferProvider) { this.queues = queues; this.bufferProvider = bufferProvider; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java index b5d9da0f1a..3975a71f72 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java @@ -57,6 +57,7 @@ public class BufferBuilderAndConsumerTest { @Test public void append() { BufferBuilder bufferBuilder = createBufferBuilder(); + BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); int[] intsToWrite = new int[] {0, 1, 2, 3, 42}; ByteBuffer bytesToWrite = toByteBuffer(intsToWrite); @@ -66,7 +67,7 @@ public class BufferBuilderAndConsumerTest { assertEquals(bytesToWrite.limit(), bytesToWrite.position()); assertFalse(bufferBuilder.isFull()); - assertContent(bufferBuilder.createBufferConsumer(), intsToWrite); + assertContent(bufferConsumer, intsToWrite); } @Test @@ -116,11 +117,21 @@ public class BufferBuilderAndConsumerTest { assertContent(bufferConsumer, 42); } - @Test(expected = IllegalStateException.class) + @Test public void creatingBufferConsumerTwice() { BufferBuilder bufferBuilder = createBufferBuilder(); - bufferBuilder.createBufferConsumer(); - bufferBuilder.createBufferConsumer(); + BufferConsumer bufferConsumer1 = bufferBuilder.createBufferConsumer(); + + assertEquals(0, bufferConsumer1.getCurrentReaderPosition()); + assertContent(bufferConsumer1); + + ByteBuffer bytesToWrite = toByteBuffer(0, 1); + bufferBuilder.appendAndCommit(bytesToWrite); + BufferConsumer bufferConsumer2 = bufferBuilder.createBufferConsumer(); + bufferBuilder.appendAndCommit(toByteBuffer(2)); + + assertEquals(bytesToWrite.position(), bufferConsumer2.getCurrentReaderPosition()); + assertContent(bufferConsumer2, 2); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java index 7696e08ad8..331854c4b8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java @@ -41,10 +41,6 @@ public class BufferBuilderTestUtils { return createFilledBufferBuilder(size, 0); } - public static BufferBuilder createFilledBufferBuilder(int dataSize) { - return createFilledBufferBuilder(BUFFER_SIZE, dataSize); - } - public static BufferBuilder createFilledBufferBuilder(int size, int dataSize) { checkArgument(size >= dataSize); BufferBuilder bufferBuilder = new BufferBuilder( @@ -70,14 +66,26 @@ public class BufferBuilderTestUtils { return buffer; } - public static BufferConsumer createFilledBufferConsumer(int size, int dataSize) { - BufferBuilder bufferBuilder = createFilledBufferBuilder(size, dataSize); - bufferBuilder.finish(); - return bufferBuilder.createBufferConsumer(); + public static BufferConsumer createFilledFinishedBufferConsumer(int dataSize) { + return createFilledBufferConsumer(dataSize, dataSize, true); } - public static BufferConsumer createFilledBufferConsumer(int dataSize) { - return createFilledBufferConsumer(BUFFER_SIZE, dataSize); + public static BufferConsumer createFilledUnfinishedBufferConsumer(int dataSize) { + return createFilledBufferConsumer(dataSize, dataSize, false); + } + + public static BufferConsumer createFilledBufferConsumer(int size, int dataSize, boolean isFinished) { + checkArgument(size >= dataSize); + + BufferBuilder bufferBuilder = createBufferBuilder(size); + BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); + fillBufferBuilder(bufferBuilder, dataSize); + + if (isFinished) { + bufferBuilder.finish(); + } + + return bufferConsumer; } public static BufferConsumer createEventBufferConsumer(int size) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionAvailabilityTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionAvailabilityTest.java index 915cf43fae..1110ccad53 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionAvailabilityTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionAvailabilityTest.java @@ -134,7 +134,7 @@ public class BoundedBlockingSubpartitionAvailabilityTest { private static void writeBuffers(ResultSubpartition partition, int numberOfBuffers) throws IOException { for (int i = 0; i < numberOfBuffers; i++) { - partition.add(BufferBuilderTestUtils.createFilledBufferConsumer(BUFFER_SIZE, BUFFER_SIZE)); + partition.add(BufferBuilderTestUtils.createFilledFinishedBufferConsumer(BUFFER_SIZE)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelBoundedDataTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelBoundedDataTest.java index 1ca2bc8eeb..c088d2542b 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelBoundedDataTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelBoundedDataTest.java @@ -32,7 +32,7 @@ import java.io.IOException; import java.nio.file.Path; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.buildSomeBuffer; -import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer; +import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; @@ -173,7 +173,7 @@ public class FileChannelBoundedDataTest extends BoundedDataTestBase { private static void writeBuffers(ResultSubpartition subpartition, int numberOfBuffers) throws IOException { for (int i = 0; i < numberOfBuffers; i++) { - subpartition.add(createFilledBufferConsumer(BUFFER_SIZE, BUFFER_SIZE)); + subpartition.add(createFilledFinishedBufferConsumer(BUFFER_SIZE)); } subpartition.finish(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java index da05f83954..45e3348157 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java @@ -47,7 +47,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.Optional; -import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer; +import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createDummyConnectionManager; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createLocalInputChannel; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createResultPartitionManager; @@ -68,7 +68,7 @@ public class InputGateFairnessTest { final int buffersPerChannel = 27; final ResultPartition resultPartition = mock(ResultPartition.class); - final BufferConsumer bufferConsumer = createFilledBufferConsumer(42); + final BufferConsumer bufferConsumer = createFilledFinishedBufferConsumer(42); // ----- create some source channels and fill them with buffers ----- @@ -122,7 +122,7 @@ public class InputGateFairnessTest { final int buffersPerChannel = 27; final ResultPartition resultPartition = mock(ResultPartition.class); - try (BufferConsumer bufferConsumer = createFilledBufferConsumer(42)) { + try (BufferConsumer bufferConsumer = createFilledFinishedBufferConsumer(42)) { // ----- create some source channels and fill them with one buffer each ----- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java index 42b0b1a838..a29f50f197 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java @@ -31,7 +31,7 @@ import org.hamcrest.Matchers; import java.io.IOException; -import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer; +import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; @@ -117,7 +117,7 @@ public enum PartitionTestUtils { int numberOfBuffers, int bufferSize) throws IOException { for (int i = 0; i < numberOfBuffers; i++) { - partition.addBufferConsumer(createFilledBufferConsumer(bufferSize, bufferSize), 0); + partition.addBufferConsumer(createFilledFinishedBufferConsumer(bufferSize), 0); } partition.finish(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index ff15b420c9..21e085d707 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -46,7 +46,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; -import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer; +import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; import static org.apache.flink.util.Preconditions.checkState; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -160,6 +160,7 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase { } final BufferBuilder bufferBuilder = bufferProvider.requestBufferBuilderBlocking(); + final BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); int segmentSize = bufferBuilder.getMaxCapacity(); MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(segmentSize); @@ -176,7 +177,7 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase { numberOfBuffers++; - return new BufferConsumerAndChannel(bufferBuilder.createBufferConsumer(), 0); + return new BufferConsumerAndChannel(bufferConsumer, 0); } }; @@ -250,8 +251,8 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase { private void testCleanupReleasedPartition(boolean createView) throws Exception { PipelinedSubpartition partition = createSubpartition(); - BufferConsumer buffer1 = createFilledBufferConsumer(4096); - BufferConsumer buffer2 = createFilledBufferConsumer(4096); + BufferConsumer buffer1 = createFilledFinishedBufferConsumer(4096); + BufferConsumer buffer2 = createFilledFinishedBufferConsumer(4096); boolean buffer1Recycled; boolean buffer2Recycled; try { @@ -306,7 +307,7 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase { private void verifyViewReleasedAfterParentRelease(ResultSubpartition partition) throws Exception { // Add a bufferConsumer - BufferConsumer bufferConsumer = createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE); + BufferConsumer bufferConsumer = createFilledFinishedBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE); partition.add(bufferConsumer); partition.finish(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java index 56945016ca..26d3f76052 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java @@ -31,12 +31,11 @@ import org.junit.Test; import javax.annotation.Nullable; import java.io.IOException; -import java.nio.ByteBuffer; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createBufferBuilder; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createEventBufferConsumer; -import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferBuilder; -import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer; +import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; +import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledUnfinishedBufferConsumer; import static org.apache.flink.runtime.io.network.util.TestBufferFactory.BUFFER_SIZE; import static org.apache.flink.util.Preconditions.checkArgument; import static org.hamcrest.MatcherAssert.assertThat; @@ -106,9 +105,7 @@ public class PipelinedSubpartitionWithReadViewTest { public void testAddNonEmptyNotFinishedBuffer() throws Exception { assertEquals(0, availablityListener.getNumNotifications()); - BufferBuilder bufferBuilder = createBufferBuilder(); - bufferBuilder.appendAndCommit(ByteBuffer.allocate(1024)); - subpartition.add(bufferBuilder.createBufferConsumer()); + subpartition.add(createFilledUnfinishedBufferConsumer(1024)); // note that since the buffer builder is not finished, there is still a retained instance! assertEquals(0, subpartition.getBuffersInBacklog()); @@ -121,8 +118,8 @@ public class PipelinedSubpartitionWithReadViewTest { */ @Test public void testUnfinishedBufferBehindFinished() throws Exception { - subpartition.add(createFilledBufferConsumer(1025)); // finished - subpartition.add(createFilledBufferBuilder(1024).createBufferConsumer()); // not finished + subpartition.add(createFilledFinishedBufferConsumer(1025)); // finished + subpartition.add(createFilledUnfinishedBufferConsumer(1024)); // not finished assertEquals(1, subpartition.getBuffersInBacklog()); assertThat(availablityListener.getNumNotifications(), greaterThan(0L)); @@ -138,8 +135,8 @@ public class PipelinedSubpartitionWithReadViewTest { */ @Test public void testFlushWithUnfinishedBufferBehindFinished() throws Exception { - subpartition.add(createFilledBufferConsumer(1025)); // finished - subpartition.add(createFilledBufferBuilder(1024).createBufferConsumer()); // not finished + subpartition.add(createFilledFinishedBufferConsumer(1025)); // finished + subpartition.add(createFilledUnfinishedBufferConsumer(1024)); // not finished long oldNumNotifications = availablityListener.getNumNotifications(); assertEquals(1, subpartition.getBuffersInBacklog()); @@ -164,8 +161,8 @@ public class PipelinedSubpartitionWithReadViewTest { subpartition.flush(); assertEquals(0, availablityListener.getNumNotifications()); - subpartition.add(createFilledBufferConsumer(1025)); // finished - subpartition.add(createFilledBufferBuilder(1024).createBufferConsumer()); // not finished + subpartition.add(createFilledFinishedBufferConsumer(1025)); // finished + subpartition.add(createFilledUnfinishedBufferConsumer(1024)); // not finished assertEquals(1, subpartition.getBuffersInBacklog()); assertNextBuffer(readView, 1025, false, 0, false, true); @@ -187,18 +184,18 @@ public class PipelinedSubpartitionWithReadViewTest { public void testMultipleEmptyBuffers() throws Exception { assertEquals(0, availablityListener.getNumNotifications()); - subpartition.add(createFilledBufferConsumer(0)); + subpartition.add(createFilledFinishedBufferConsumer(0)); + assertEquals(0, availablityListener.getNumNotifications()); + subpartition.add(createFilledFinishedBufferConsumer(0)); assertEquals(1, availablityListener.getNumNotifications()); - subpartition.add(createFilledBufferConsumer(0)); - assertEquals(2, availablityListener.getNumNotifications()); - subpartition.add(createFilledBufferConsumer(0)); - assertEquals(2, availablityListener.getNumNotifications()); + subpartition.add(createFilledFinishedBufferConsumer(0)); + assertEquals(1, availablityListener.getNumNotifications()); assertEquals(2, subpartition.getBuffersInBacklog()); - subpartition.add(createFilledBufferConsumer(1024)); - assertEquals(2, availablityListener.getNumNotifications()); + subpartition.add(createFilledFinishedBufferConsumer(1024)); + assertEquals(1, availablityListener.getNumNotifications()); assertNextBuffer(readView, 1024, false, 0, false, true); } @@ -218,15 +215,14 @@ public class PipelinedSubpartitionWithReadViewTest { assertEquals(0, availablityListener.getNumNotifications()); // Add data to the queue... - subpartition.add(createFilledBufferConsumer(BUFFER_SIZE)); + subpartition.add(createFilledFinishedBufferConsumer(BUFFER_SIZE)); assertFalse(readView.nextBufferIsEvent()); assertEquals(1, subpartition.getTotalNumberOfBuffers()); assertEquals(0, subpartition.getBuffersInBacklog()); assertEquals(0, subpartition.getTotalNumberOfBytes()); // only updated when getting the buffer - // ...should have resulted in a notification - assertEquals(1, availablityListener.getNumNotifications()); + assertEquals(0, availablityListener.getNumNotifications()); // ...and one available result assertNextBuffer(readView, BUFFER_SIZE, false, 0, false, true); @@ -236,13 +232,13 @@ public class PipelinedSubpartitionWithReadViewTest { assertEquals(0, subpartition.getBuffersInBacklog()); // Add data to the queue... - subpartition.add(createFilledBufferConsumer(BUFFER_SIZE)); + subpartition.add(createFilledFinishedBufferConsumer(BUFFER_SIZE)); assertFalse(readView.nextBufferIsEvent()); assertEquals(2, subpartition.getTotalNumberOfBuffers()); assertEquals(0, subpartition.getBuffersInBacklog()); assertEquals(BUFFER_SIZE, subpartition.getTotalNumberOfBytes()); // only updated when getting the buffer - assertEquals(2, availablityListener.getNumNotifications()); + assertEquals(0, availablityListener.getNumNotifications()); assertNextBuffer(readView, BUFFER_SIZE, false, 0, false, true); assertEquals(2 * BUFFER_SIZE, subpartition.getTotalNumberOfBytes()); // only updated when getting the buffer @@ -253,17 +249,17 @@ public class PipelinedSubpartitionWithReadViewTest { // some tests with events // fill with: buffer, event, and buffer - subpartition.add(createFilledBufferConsumer(BUFFER_SIZE)); + subpartition.add(createFilledFinishedBufferConsumer(BUFFER_SIZE)); assertFalse(readView.nextBufferIsEvent()); subpartition.add(createEventBufferConsumer(BUFFER_SIZE)); assertFalse(readView.nextBufferIsEvent()); - subpartition.add(createFilledBufferConsumer(BUFFER_SIZE)); + subpartition.add(createFilledFinishedBufferConsumer(BUFFER_SIZE)); assertFalse(readView.nextBufferIsEvent()); assertEquals(5, subpartition.getTotalNumberOfBuffers()); assertEquals(1, subpartition.getBuffersInBacklog()); // two buffers (events don't count) assertEquals(2 * BUFFER_SIZE, subpartition.getTotalNumberOfBytes()); // only updated when getting the buffer - assertEquals(4, availablityListener.getNumNotifications()); + assertEquals(1, availablityListener.getNumNotifications()); // the first buffer assertNextBuffer(readView, BUFFER_SIZE, true, 0, true, true); @@ -271,7 +267,7 @@ public class PipelinedSubpartitionWithReadViewTest { assertEquals(0, subpartition.getBuffersInBacklog()); // the event - assertNextEvent(readView, BUFFER_SIZE, null, true, 0, false, true); + assertNextEvent(readView, BUFFER_SIZE, null, false, 0, false, true); assertEquals(4 * BUFFER_SIZE, subpartition.getTotalNumberOfBytes()); // only updated when getting the buffer assertEquals(0, subpartition.getBuffersInBacklog()); @@ -286,7 +282,7 @@ public class PipelinedSubpartitionWithReadViewTest { assertEquals(5, subpartition.getTotalNumberOfBuffers()); assertEquals(5 * BUFFER_SIZE, subpartition.getTotalNumberOfBytes()); - assertEquals(4, availablityListener.getNumNotifications()); + assertEquals(1, availablityListener.getNumNotifications()); } @Test @@ -308,11 +304,10 @@ public class PipelinedSubpartitionWithReadViewTest { final int numberOfAddedBuffers = 5; for (int i = 1; i <= numberOfAddedBuffers; i++) { - final BufferBuilder bufferBuilder = createFilledBufferBuilder(BUFFER_SIZE); - subpartition.add(bufferBuilder.createBufferConsumer()); - if (i < numberOfAddedBuffers || isFinished) { - bufferBuilder.finish(); + subpartition.add(createFilledFinishedBufferConsumer(1024)); + } else { + subpartition.add(createFilledUnfinishedBufferConsumer(1024)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java index dc693e53a9..0b1e9a0c98 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java @@ -40,7 +40,7 @@ import org.junit.Test; import java.io.IOException; import java.util.Collections; -import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer; +import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createPartition; import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.verifyCreateSubpartitionViewThrowsException; import static org.hamcrest.MatcherAssert.assertThat; @@ -90,7 +90,7 @@ public class ResultPartitionTest { taskActions, jobId, notifier); - consumableNotifyingPartitionWriter.addBufferConsumer(createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE), 0); + consumableNotifyingPartitionWriter.addBufferConsumer(createFilledFinishedBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE), 0); verify(notifier, times(1)) .notifyPartitionConsumable(eq(jobId), eq(consumableNotifyingPartitionWriter.getPartitionId()), eq(taskActions)); } @@ -103,7 +103,7 @@ public class ResultPartitionTest { taskActions, jobId, notifier); - partition.addBufferConsumer(createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE), 0); + partition.addBufferConsumer(createFilledFinishedBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE), 0); verify(notifier, never()).notifyPartitionConsumable(eq(jobId), eq(partition.getPartitionId()), eq(taskActions)); } } @@ -151,7 +151,7 @@ public class ResultPartitionTest { * @param partitionType the result partition type to set up */ private void testAddOnFinishedPartition(final ResultPartitionType partitionType) throws Exception { - BufferConsumer bufferConsumer = createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE); + BufferConsumer bufferConsumer = createFilledFinishedBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE); ResultPartitionConsumableNotifier notifier = mock(ResultPartitionConsumableNotifier.class); JobID jobId = new JobID(); TaskActions taskActions = new NoOpTaskActions(); @@ -197,7 +197,7 @@ public class ResultPartitionTest { * @param partitionType the result partition type to set up */ private void testAddOnReleasedPartition(final ResultPartitionType partitionType) throws Exception { - BufferConsumer bufferConsumer = createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE); + BufferConsumer bufferConsumer = createFilledFinishedBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE); ResultPartitionConsumableNotifier notifier = mock(ResultPartitionConsumableNotifier.class); JobID jobId = new JobID(); TaskActions taskActions = new NoOpTaskActions(); @@ -267,7 +267,7 @@ public class ResultPartitionTest { taskActions, jobId, notifier); - BufferConsumer bufferConsumer = createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE); + BufferConsumer bufferConsumer = createFilledFinishedBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE); try { // partition.add() adds the bufferConsumer without recycling it (if not spilling) consumableNotifyingPartitionWriter.addBufferConsumer(bufferConsumer, 0); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java index 13cab65bb9..2fef1828e0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java @@ -24,7 +24,7 @@ import org.apache.flink.util.TestLogger; import org.junit.Test; -import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer; +import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -70,7 +70,7 @@ public abstract class SubpartitionTestBase extends TestLogger { assertEquals(1, subpartition.getTotalNumberOfBuffers()); assertEquals(0, subpartition.getBuffersInBacklog()); - BufferConsumer bufferConsumer = createFilledBufferConsumer(4096, 4096); + BufferConsumer bufferConsumer = createFilledFinishedBufferConsumer(4096); assertFalse(subpartition.add(bufferConsumer)); assertTrue(bufferConsumer.isRecycled()); @@ -91,7 +91,7 @@ public abstract class SubpartitionTestBase extends TestLogger { try { subpartition.release(); - BufferConsumer bufferConsumer = createFilledBufferConsumer(4096, 4096); + BufferConsumer bufferConsumer = createFilledFinishedBufferConsumer(4096); assertFalse(subpartition.add(bufferConsumer)); assertTrue(bufferConsumer.isRecycled()); @@ -106,7 +106,7 @@ public abstract class SubpartitionTestBase extends TestLogger { @Test public void testReleasingReaderDoesNotReleasePartition() throws Exception { final ResultSubpartition partition = createSubpartition(); - partition.add(createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE)); + partition.add(createFilledFinishedBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE)); partition.finish(); final ResultSubpartitionView reader = partition.createReadView(new NoOpBufferAvailablityListener()); @@ -125,7 +125,7 @@ public abstract class SubpartitionTestBase extends TestLogger { @Test public void testReleaseIsIdempotent() throws Exception { final ResultSubpartition partition = createSubpartition(); - partition.add(createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE)); + partition.add(createFilledFinishedBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE)); partition.finish(); partition.release(); @@ -136,7 +136,7 @@ public abstract class SubpartitionTestBase extends TestLogger { @Test public void testReadAfterDispose() throws Exception { final ResultSubpartition partition = createSubpartition(); - partition.add(createFilledBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE)); + partition.add(createFilledFinishedBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE)); partition.finish(); final ResultSubpartitionView reader = partition.createReadView(new NoOpBufferAvailablityListener()); @@ -154,7 +154,7 @@ public abstract class SubpartitionTestBase extends TestLogger { public void testRecycleBufferAndConsumerOnFailure() throws Exception { final ResultSubpartition subpartition = createFailingWritesSubpartition(); try { - final BufferConsumer consumer = BufferBuilderTestUtils.createFilledBufferConsumer(100); + final BufferConsumer consumer = BufferBuilderTestUtils.createFilledFinishedBufferConsumer(100); try { subpartition.add(consumer); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java index d691f3e22d..4cbc6c8b23 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer; import org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability; import org.apache.flink.runtime.io.network.partition.consumer.TestInputChannel.BufferAndAvailabilityProvider; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; @@ -33,7 +34,6 @@ import org.apache.flink.util.MutableObjectIterator; import java.io.IOException; import java.util.Optional; -import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.buildSingleBuffer; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createBufferBuilder; /** @@ -77,12 +77,13 @@ public class IteratorWrappingTestSingleInputGate e if (hasData) { serializer.serializeRecord(reuse); BufferBuilder bufferBuilder = createBufferBuilder(bufferSize); + BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); serializer.copyToBufferBuilder(bufferBuilder); hasData = inputIterator.next(reuse) != null; // Call getCurrentBuffer to ensure size is set - return Optional.of(new BufferAndAvailability(buildSingleBuffer(bufferBuilder), true, 0)); + return Optional.of(new BufferAndAvailability(bufferConsumer.build(), true, 0)); } else { inputChannel.setReleased(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java index dc8b501d87..d049f9b0a7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; @@ -465,9 +466,10 @@ public class LocalInputChannelTest { if (channelIndexes.size() > 0) { final int channelIndex = channelIndexes.remove(0); BufferBuilder bufferBuilder = bufferProvider.requestBufferBuilderBlocking(); + BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); bufferBuilder.appendAndCommit(ByteBuffer.wrap(new byte[4])); bufferBuilder.finish(); - return new BufferConsumerAndChannel(bufferBuilder.createBufferConsumer(), channelIndex); + return new BufferConsumerAndChannel(bufferConsumer, channelIndex); } return null; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index 3a8e4f23ef..57f0a17b23 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -594,7 +594,7 @@ public class SingleInputGateTest extends InputGateTestBase { remoteInputChannel.onBuffer(TestBufferFactory.createBuffer(1), 0, 0); assertEquals(1, inputGate.getNumberOfQueuedBuffers()); - resultPartition.addBufferConsumer(BufferBuilderTestUtils.createFilledBufferConsumer(1), 0); + resultPartition.addBufferConsumer(BufferBuilderTestUtils.createFilledFinishedBufferConsumer(1), 0); assertEquals(2, inputGate.getNumberOfQueuedBuffers()); } finally { resultPartition.release(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java index 291e15ccd6..d3754fde19 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer; import org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability; import org.apache.flink.runtime.io.network.partition.consumer.TestInputChannel.BufferAndAvailabilityProvider; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; @@ -38,7 +39,6 @@ import java.io.IOException; import java.util.Optional; import java.util.concurrent.ConcurrentLinkedQueue; -import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.buildSingleBuffer; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createBufferBuilder; /** @@ -106,11 +106,12 @@ public class StreamTestSingleInputGate extends TestSingleInputGate { delegate.setInstance(inputElement); recordSerializer.serializeRecord(delegate); BufferBuilder bufferBuilder = createBufferBuilder(bufferSize); + BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); recordSerializer.copyToBufferBuilder(bufferBuilder); bufferBuilder.finish(); // Call getCurrentBuffer to ensure size is set - return Optional.of(new BufferAndAvailability(buildSingleBuffer(bufferBuilder), moreAvailable, 0)); + return Optional.of(new BufferAndAvailability(bufferConsumer.build(), moreAvailable, 0)); } else if (input != null && input.isEvent()) { AbstractEvent event = input.getEvent(); if (event instanceof EndOfPartitionEvent) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java index 17b9e063e8..d4ba08ff93 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/StreamTaskNetworkInputTest.java @@ -25,9 +25,9 @@ import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer; import org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer; import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer; -import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate; import org.apache.flink.runtime.plugable.DeserializationDelegate; @@ -74,13 +74,12 @@ public class StreamTaskNetworkInputTest { @Test public void testIsAvailableWithBufferedDataInDeserializer() throws Exception { BufferBuilder bufferBuilder = BufferBuilderTestUtils.createEmptyBufferBuilder(PAGE_SIZE); + BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); serializeRecord(42L, bufferBuilder); serializeRecord(44L, bufferBuilder); - Buffer buffer = bufferBuilder.createBufferConsumer().build(); - - List buffers = Collections.singletonList(new BufferOrEvent(buffer, 0, false)); + List buffers = Collections.singletonList(new BufferOrEvent(bufferConsumer.build(), 0, false)); VerifyRecordsDataOutput output = new VerifyRecordsDataOutput<>(); StreamTaskNetworkInput input = new StreamTaskNetworkInput<>( -- Gitee From 9373126f7d62f01a203e2d7c2eaadb7d2865b178 Mon Sep 17 00:00:00 2001 From: vthinkxie Date: Sat, 27 Jul 2019 15:02:59 +0800 Subject: [PATCH 066/268] [FLINK-13386][web]: Fix job manager configuration sort --- .../web-dashboard/package-lock.json | 77 ++++++++++++++----- .../job-manager-configuration.component.ts | 2 +- 2 files changed, 59 insertions(+), 20 deletions(-) diff --git a/flink-runtime-web/web-dashboard/package-lock.json b/flink-runtime-web/web-dashboard/package-lock.json index 42e95ae537..b9d70d0f6e 100644 --- a/flink-runtime-web/web-dashboard/package-lock.json +++ b/flink-runtime-web/web-dashboard/package-lock.json @@ -1572,6 +1572,7 @@ "resolved": "https://registry.npmjs.org/are-we-there-yet/-/are-we-there-yet-1.1.5.tgz", "integrity": "sha512-5hYdAkZlcG8tOLujVDTgCT+uPX0VnpAH28gWsLfzpXYm7wP6mp5Q/gYyR7YQ0cKVJcXJnl3j2kpBan13PtQf6w==", "dev": true, + "optional": true, "requires": { "delegates": "^1.0.0", "readable-stream": "^2.0.6" @@ -2806,7 +2807,8 @@ "version": "1.1.0", "resolved": "https://registry.npmjs.org/console-control-strings/-/console-control-strings-1.1.0.tgz", "integrity": "sha1-PXz0Rk22RG6mRL9LOVB/mFEAjo4=", - "dev": true + "dev": true, + "optional": true }, "constants-browserify": { "version": "1.0.0", @@ -3508,7 +3510,8 @@ "version": "1.0.0", "resolved": "https://registry.npmjs.org/delegates/-/delegates-1.0.0.tgz", "integrity": "sha1-hMbhWbgZBP3KWaDvRM2HDTElD5o=", - "dev": true + "dev": true, + "optional": true }, "depd": { "version": "1.1.2", @@ -4505,7 +4508,8 @@ "ansi-regex": { "version": "2.1.1", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "aproba": { "version": "1.2.0", @@ -4526,12 +4530,14 @@ "balanced-match": { "version": "1.0.0", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "brace-expansion": { "version": "1.1.11", "bundled": true, "dev": true, + "optional": true, "requires": { "balanced-match": "^1.0.0", "concat-map": "0.0.1" @@ -4546,17 +4552,20 @@ "code-point-at": { "version": "1.1.0", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "concat-map": { "version": "0.0.1", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "console-control-strings": { "version": "1.1.0", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "core-util-is": { "version": "1.0.2", @@ -4673,7 +4682,8 @@ "inherits": { "version": "2.0.3", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "ini": { "version": "1.3.5", @@ -4685,6 +4695,7 @@ "version": "1.0.0", "bundled": true, "dev": true, + "optional": true, "requires": { "number-is-nan": "^1.0.0" } @@ -4699,6 +4710,7 @@ "version": "3.0.4", "bundled": true, "dev": true, + "optional": true, "requires": { "brace-expansion": "^1.1.7" } @@ -4706,12 +4718,14 @@ "minimist": { "version": "0.0.8", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "minipass": { "version": "2.3.5", "bundled": true, "dev": true, + "optional": true, "requires": { "safe-buffer": "^5.1.2", "yallist": "^3.0.0" @@ -4730,6 +4744,7 @@ "version": "0.5.1", "bundled": true, "dev": true, + "optional": true, "requires": { "minimist": "0.0.8" } @@ -4810,7 +4825,8 @@ "number-is-nan": { "version": "1.0.1", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "object-assign": { "version": "4.1.1", @@ -4822,6 +4838,7 @@ "version": "1.4.0", "bundled": true, "dev": true, + "optional": true, "requires": { "wrappy": "1" } @@ -4907,7 +4924,8 @@ "safe-buffer": { "version": "5.1.2", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "safer-buffer": { "version": "2.1.2", @@ -4943,6 +4961,7 @@ "version": "1.0.2", "bundled": true, "dev": true, + "optional": true, "requires": { "code-point-at": "^1.0.0", "is-fullwidth-code-point": "^1.0.0", @@ -4962,6 +4981,7 @@ "version": "3.0.1", "bundled": true, "dev": true, + "optional": true, "requires": { "ansi-regex": "^2.0.0" } @@ -5005,12 +5025,14 @@ "wrappy": { "version": "1.0.2", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "yallist": { "version": "3.0.3", "bundled": true, - "dev": true + "dev": true, + "optional": true } } }, @@ -5019,6 +5041,7 @@ "resolved": "https://registry.npmjs.org/fstream/-/fstream-1.0.11.tgz", "integrity": "sha1-XB+x8RdHcRTwYyoOtLcbPLD9MXE=", "dev": true, + "optional": true, "requires": { "graceful-fs": "^4.1.2", "inherits": "~2.0.0", @@ -5047,6 +5070,7 @@ "resolved": "https://registry.npmjs.org/gauge/-/gauge-2.7.4.tgz", "integrity": "sha1-LANAXHU4w51+s3sxcCLjJfsBi/c=", "dev": true, + "optional": true, "requires": { "aproba": "^1.0.3", "console-control-strings": "^1.0.0", @@ -5090,7 +5114,8 @@ "version": "4.0.1", "resolved": "https://registry.npmjs.org/get-stdin/-/get-stdin-4.0.1.tgz", "integrity": "sha1-uWjGsKBDhDJJAui/Gl3zJXmkUP4=", - "dev": true + "dev": true, + "optional": true }, "get-stream": { "version": "3.0.0", @@ -5300,7 +5325,8 @@ "version": "2.0.1", "resolved": "https://registry.npmjs.org/has-unicode/-/has-unicode-2.0.1.tgz", "integrity": "sha1-4Ob+aijPUROIVeCG0Wkedx3iqLk=", - "dev": true + "dev": true, + "optional": true }, "has-value": { "version": "1.0.0", @@ -6252,7 +6278,8 @@ "version": "0.2.1", "resolved": "https://registry.npmjs.org/is-utf8/-/is-utf8-0.2.1.tgz", "integrity": "sha1-Sw2hRCEE0bM2NA6AeX6GXPOffXI=", - "dev": true + "dev": true, + "optional": true }, "is-windows": { "version": "1.0.2", @@ -7135,6 +7162,7 @@ "resolved": "https://registry.npmjs.org/load-json-file/-/load-json-file-1.1.0.tgz", "integrity": "sha1-lWkFcI1YtLq0wiYbBPWfMcmTdMA=", "dev": true, + "optional": true, "requires": { "graceful-fs": "^4.1.2", "parse-json": "^2.2.0", @@ -7147,7 +7175,8 @@ "version": "2.3.0", "resolved": "https://registry.npmjs.org/pify/-/pify-2.3.0.tgz", "integrity": "sha1-7RQaasBDqEnqWISY59yosVMw6Qw=", - "dev": true + "dev": true, + "optional": true } } }, @@ -7520,7 +7549,8 @@ "version": "1.0.1", "resolved": "https://registry.npmjs.org/map-obj/-/map-obj-1.0.1.tgz", "integrity": "sha1-2TPOuSBdgr3PSIb2dCvcK03qFG0=", - "dev": true + "dev": true, + "optional": true }, "map-visit": { "version": "1.0.0", @@ -8170,6 +8200,7 @@ "resolved": "https://registry.npmjs.org/npmlog/-/npmlog-4.1.2.tgz", "integrity": "sha512-2uUqazuKlTaSI/dC8AzicUck7+IrEaOnN/e0jd3Xtt1KcGpwx30v50mL7oPyr/h9bL3E4aZccVwpwP+5W9Vjkg==", "dev": true, + "optional": true, "requires": { "are-we-there-yet": "~1.1.2", "console-control-strings": "~1.1.0", @@ -9217,6 +9248,7 @@ "resolved": "https://registry.npmjs.org/read-pkg/-/read-pkg-1.1.0.tgz", "integrity": "sha1-9f+qXs0pyzHAR0vKfXVra7KePyg=", "dev": true, + "optional": true, "requires": { "load-json-file": "^1.0.0", "normalize-package-data": "^2.3.2", @@ -9228,6 +9260,7 @@ "resolved": "https://registry.npmjs.org/path-type/-/path-type-1.1.0.tgz", "integrity": "sha1-WcRPfuSR2nBNpBXaWkBwuk+P5EE=", "dev": true, + "optional": true, "requires": { "graceful-fs": "^4.1.2", "pify": "^2.0.0", @@ -9238,7 +9271,8 @@ "version": "2.3.0", "resolved": "https://registry.npmjs.org/pify/-/pify-2.3.0.tgz", "integrity": "sha1-7RQaasBDqEnqWISY59yosVMw6Qw=", - "dev": true + "dev": true, + "optional": true } } }, @@ -9247,6 +9281,7 @@ "resolved": "https://registry.npmjs.org/read-pkg-up/-/read-pkg-up-1.0.1.tgz", "integrity": "sha1-nWPBMnbAZZGNV/ACpX9AobZD+wI=", "dev": true, + "optional": true, "requires": { "find-up": "^1.0.0", "read-pkg": "^1.0.0" @@ -9257,6 +9292,7 @@ "resolved": "https://registry.npmjs.org/find-up/-/find-up-1.1.2.tgz", "integrity": "sha1-ay6YIrGizgpgq2TWEOzK1TyyTQ8=", "dev": true, + "optional": true, "requires": { "path-exists": "^2.0.0", "pinkie-promise": "^2.0.0" @@ -9267,6 +9303,7 @@ "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-2.1.0.tgz", "integrity": "sha1-D+tsZPD8UY2adU3V77YscCJ2H0s=", "dev": true, + "optional": true, "requires": { "pinkie-promise": "^2.0.0" } @@ -10719,6 +10756,7 @@ "resolved": "https://registry.npmjs.org/strip-bom/-/strip-bom-2.0.0.tgz", "integrity": "sha1-YhmoVhZSBJHzV4i9vxRHqZx+aw4=", "dev": true, + "optional": true, "requires": { "is-utf8": "^0.2.0" } @@ -12139,6 +12177,7 @@ "resolved": "https://registry.npmjs.org/wide-align/-/wide-align-1.1.3.tgz", "integrity": "sha512-QGkOQc8XL6Bt5PwnsExKBPuMKBxnGxWWW3fU55Xt4feHozMUhdUMaBCk290qpm/wG5u/RSKzwdAC4i51YigihA==", "dev": true, + "optional": true, "requires": { "string-width": "^1.0.2 || 2" } diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/configuration/job-manager-configuration.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/configuration/job-manager-configuration.component.ts index e130ed2bdd..1dc88ee0aa 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/configuration/job-manager-configuration.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/configuration/job-manager-configuration.component.ts @@ -32,7 +32,7 @@ export class JobManagerConfigurationComponent implements OnInit { ngOnInit() { this.jobManagerService.loadConfig().subscribe(data => { - this.listOfConfig = data; + this.listOfConfig = data.sort((pre, next) => (pre.key > next.key ? 1 : -1)); this.cdr.markForCheck(); }); } -- Gitee From fe254a3b27ffd6d210de5e2af6e8dab4a39f3211 Mon Sep 17 00:00:00 2001 From: vthinkxie Date: Sat, 27 Jul 2019 15:33:22 +0800 Subject: [PATCH 067/268] [FLINK-13386][web]: Fix job subtask sort --- flink-runtime-web/web-dashboard/src/app/services/job.service.ts | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-runtime-web/web-dashboard/src/app/services/job.service.ts b/flink-runtime-web/web-dashboard/src/app/services/job.service.ts index 4da2b70342..48f32a1dcd 100644 --- a/flink-runtime-web/web-dashboard/src/app/services/job.service.ts +++ b/flink-runtime-web/web-dashboard/src/app/services/job.service.ts @@ -264,6 +264,8 @@ export class JobService { }); } }); + const listOfVerticesId = job.vertices.map(item => item.id); + nodes.sort((pre, next) => listOfVerticesId.indexOf(next.id) - listOfVerticesId.indexOf(pre.id)); } return { ...job, -- Gitee From 0078e7a2a9aca2b0e430d265b12cf946ae34aec7 Mon Sep 17 00:00:00 2001 From: vthinkxie Date: Sat, 27 Jul 2019 15:58:48 +0800 Subject: [PATCH 068/268] [FLINK-13386][web]: Fix operators/tasks metrics sort --- .../src/app/services/metrics.service.ts | 20 ++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/flink-runtime-web/web-dashboard/src/app/services/metrics.service.ts b/flink-runtime-web/web-dashboard/src/app/services/metrics.service.ts index 081aa4efa2..d13b3a674a 100644 --- a/flink-runtime-web/web-dashboard/src/app/services/metrics.service.ts +++ b/flink-runtime-web/web-dashboard/src/app/services/metrics.service.ts @@ -33,9 +33,23 @@ export class MetricsService { * @param vertexId */ getAllAvailableMetrics(jobId: string, vertexId: string) { - return this.httpClient.get>( - `${BASE_URL}/jobs/${jobId}/vertices/${vertexId}/metrics` - ); + return this.httpClient + .get>(`${BASE_URL}/jobs/${jobId}/vertices/${vertexId}/metrics`) + .pipe( + map(item => + item.sort((pre, next) => { + const preId = pre.id.toLowerCase(); + const nextId = next.id.toLowerCase(); + if (preId < nextId) { + return -1; + } else if (preId > nextId) { + return 1; + } else { + return 0; + } + }) + ) + ); } /** -- Gitee From 90b4022cad3870b2b05550e33302adf185cd1587 Mon Sep 17 00:00:00 2001 From: vthinkxie Date: Sat, 27 Jul 2019 16:42:08 +0800 Subject: [PATCH 069/268] [FLINK-13386][web]: Add numeric metrics in job --- .../job-chart/job-chart.component.html | 13 ++++-- .../job-chart/job-chart.component.less | 6 +++ .../job-chart/job-chart.component.ts | 13 +++++- .../pipes/humanize-chart-numeric.pipe.ts | 46 +++++++++++++++++++ .../app/share/pipes/humanize-duration.pipe.ts | 14 ++++-- .../src/app/share/pipes/pipe.module.ts | 11 ++++- 6 files changed, 94 insertions(+), 9 deletions(-) create mode 100644 flink-runtime-web/web-dashboard/src/app/share/pipes/humanize-chart-numeric.pipe.ts diff --git a/flink-runtime-web/web-dashboard/src/app/share/customize/job-chart/job-chart.component.html b/flink-runtime-web/web-dashboard/src/app/share/customize/job-chart/job-chart.component.html index f20258eed2..4ab72d6ba2 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/customize/job-chart/job-chart.component.html +++ b/flink-runtime-web/web-dashboard/src/app/share/customize/job-chart/job-chart.component.html @@ -21,13 +21,20 @@

{{title}}
- - + +
-
+
+
{{latestValue | humanizeChartNumeric:title}}
+
+ + + + +
diff --git a/flink-runtime-web/web-dashboard/src/app/share/customize/job-chart/job-chart.component.less b/flink-runtime-web/web-dashboard/src/app/share/customize/job-chart/job-chart.component.less index 4cfe665b43..100027c336 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/customize/job-chart/job-chart.component.less +++ b/flink-runtime-web/web-dashboard/src/app/share/customize/job-chart/job-chart.component.less @@ -59,6 +59,12 @@ .content { padding: 12px; + + .numeric { + line-height: 150px; + font-size: 32px; + text-align: center; + } } button { diff --git a/flink-runtime-web/web-dashboard/src/app/share/customize/job-chart/job-chart.component.ts b/flink-runtime-web/web-dashboard/src/app/share/customize/job-chart/job-chart.component.ts index 2e33c0d69e..e5b716c3f8 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/customize/job-chart/job-chart.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/share/customize/job-chart/job-chart.component.ts @@ -45,8 +45,10 @@ export class JobChartComponent implements AfterViewInit, OnDestroy { @Output() closed = new EventEmitter(); @ViewChild('chart') chart: ElementRef; size = 'small'; + displayMode: 'chart' | 'numeric' = 'chart'; chartInstance: Chart; data: Array<{ time: number; value: number; type: string }> = []; + latestValue: number; @HostBinding('class.big') get isBig() { @@ -54,9 +56,13 @@ export class JobChartComponent implements AfterViewInit, OnDestroy { } refresh(res: { timestamp: number; values: { [id: string]: number } }) { + this.latestValue = res.values[this.title]; + if (this.displayMode === 'numeric') { + this.cdr.detectChanges(); + } this.data.push({ time: res.timestamp, - value: res.values[this.title], + value: this.latestValue, type: this.title }); @@ -68,6 +74,11 @@ export class JobChartComponent implements AfterViewInit, OnDestroy { } } + setMode(mode: 'chart' | 'numeric') { + this.displayMode = mode; + this.cdr.detectChanges(); + } + resize(size: string) { this.size = size; this.cdr.detectChanges(); diff --git a/flink-runtime-web/web-dashboard/src/app/share/pipes/humanize-chart-numeric.pipe.ts b/flink-runtime-web/web-dashboard/src/app/share/pipes/humanize-chart-numeric.pipe.ts new file mode 100644 index 0000000000..4ad8b92496 --- /dev/null +++ b/flink-runtime-web/web-dashboard/src/app/share/pipes/humanize-chart-numeric.pipe.ts @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { Pipe, PipeTransform } from '@angular/core'; +import { isNil } from 'utils'; +import { HumanizeBytesPipe } from './humanize-bytes.pipe'; +import { HumanizeDurationPipe } from './humanize-duration.pipe'; + +@Pipe({ + name: 'humanizeChartNumeric' +}) +export class HumanizeChartNumericPipe implements PipeTransform { + transform(value: number, id: string): string { + if (isNil(value)) { + return '-'; + } + let returnVal = ''; + if (/bytes/i.test(id) && /persecond/i.test(id)) { + returnVal = new HumanizeBytesPipe().transform(value) + ' / s'; + } else if (/bytes/i.test(id)) { + returnVal = new HumanizeBytesPipe().transform(value); + } else if (/persecond/i.test(id)) { + returnVal = value + ' / s'; + } else if (/time/i.test(id) || /latency/i.test(id)) { + returnVal = new HumanizeDurationPipe().transform(value, true); + } else { + returnVal = `${value}`; + } + return returnVal; + } +} diff --git a/flink-runtime-web/web-dashboard/src/app/share/pipes/humanize-duration.pipe.ts b/flink-runtime-web/web-dashboard/src/app/share/pipes/humanize-duration.pipe.ts index 7440ddf1e6..3cf8043055 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/pipes/humanize-duration.pipe.ts +++ b/flink-runtime-web/web-dashboard/src/app/share/pipes/humanize-duration.pipe.ts @@ -23,7 +23,7 @@ import { isNil } from 'utils'; name: 'humanizeDuration' }) export class HumanizeDurationPipe implements PipeTransform { - transform(value: number): any { + transform(value: number, short: boolean = false): any { if (isNil(value)) { return '-'; } else if (value < 0) { @@ -50,10 +50,18 @@ export class HumanizeDurationPipe implements PipeTransform { return `${minutes}m ${seconds}s`; } } else { - return `${hours}h ${minutes}m`; + if (short) { + return `${hours}h ${minutes}m`; + } else { + return `${hours}h ${minutes}m ${seconds}s`; + } } } else { - return `${days}d ${hours}h ${minutes}m`; + if (short) { + return `${days}d ${hours}h`; + } else { + return `${days}d ${hours}h ${minutes}m ${seconds}s`; + } } } } diff --git a/flink-runtime-web/web-dashboard/src/app/share/pipes/pipe.module.ts b/flink-runtime-web/web-dashboard/src/app/share/pipes/pipe.module.ts index 185c820553..8afe7cc667 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/pipes/pipe.module.ts +++ b/flink-runtime-web/web-dashboard/src/app/share/pipes/pipe.module.ts @@ -22,10 +22,17 @@ import { HumanizeBytesPipe } from 'share/pipes/humanize-bytes.pipe'; import { HumanizeWatermarkPipe } from 'share/pipes/humanize-watermark.pipe'; import { HumanizeDurationPipe } from './humanize-duration.pipe'; import { HumanizeDatePipe } from './humanize-date.pipe'; +import { HumanizeChartNumericPipe } from './humanize-chart-numeric.pipe'; @NgModule({ imports: [CommonModule], - declarations: [HumanizeDurationPipe, HumanizeDatePipe, HumanizeBytesPipe, HumanizeWatermarkPipe], - exports: [HumanizeDurationPipe, HumanizeDatePipe, HumanizeBytesPipe, HumanizeWatermarkPipe] + declarations: [ + HumanizeDurationPipe, + HumanizeDatePipe, + HumanizeBytesPipe, + HumanizeWatermarkPipe, + HumanizeChartNumericPipe + ], + exports: [HumanizeDurationPipe, HumanizeDatePipe, HumanizeBytesPipe, HumanizeWatermarkPipe, HumanizeChartNumericPipe] }) export class PipeModule {} -- Gitee From c46ba1bbd28782b9fe4ab5737aee367b5314a818 Mon Sep 17 00:00:00 2001 From: vthinkxie Date: Sat, 27 Jul 2019 17:56:26 +0800 Subject: [PATCH 070/268] [FLINK-13386][web]: Add all operators watermark --- .../src/app/interfaces/job-detail.ts | 1 + .../job/overview/job-overview.component.ts | 53 ++++++++++++++++--- .../share/common/dagre/node.component.html | 1 + .../share/common/dagre/node.component.less | 7 ++- .../app/share/common/dagre/node.component.ts | 2 + 5 files changed, 57 insertions(+), 7 deletions(-) diff --git a/flink-runtime-web/web-dashboard/src/app/interfaces/job-detail.ts b/flink-runtime-web/web-dashboard/src/app/interfaces/job-detail.ts index 3bf9c13e9b..14291049b5 100644 --- a/flink-runtime-web/web-dashboard/src/app/interfaces/job-detail.ts +++ b/flink-runtime-web/web-dashboard/src/app/interfaces/job-detail.ts @@ -129,6 +129,7 @@ export interface NodesItemInterface { export interface NodesItemCorrectInterface extends NodesItemInterface { detail: VerticesItemInterface | undefined; + lowWatermark?: number; } export interface NodesItemLinkInterface { diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/job-overview.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/job-overview.component.ts index 5174653f44..a0765c43df 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/job-overview.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/job-overview.component.ts @@ -26,10 +26,11 @@ import { ViewChild } from '@angular/core'; import { ActivatedRoute, Router } from '@angular/router'; -import { Subject } from 'rxjs'; -import { filter, takeUntil } from 'rxjs/operators'; +import { LONG_MIN_VALUE } from 'config'; +import { forkJoin, Observable, of, Subject } from 'rxjs'; +import { catchError, filter, map, takeUntil } from 'rxjs/operators'; import { NodesItemCorrectInterface, NodesItemLinkInterface } from 'interfaces'; -import { JobService } from 'services'; +import { JobService, MetricsService } from 'services'; import { DagreComponent } from 'share/common/dagre/dagre.component'; @Component({ @@ -62,11 +63,52 @@ export class JobOverviewComponent implements OnInit, OnDestroy { } } + mergeWithWatermarks(nodes: NodesItemCorrectInterface[]): Observable { + return forkJoin( + nodes.map(node => { + const listOfMetricId = []; + let lowWatermark = NaN; + for (let i = 0; i < node.parallelism; i++) { + listOfMetricId.push(`${i}.currentInputWatermark`); + } + return this.metricService.getMetrics(this.jobId, node.id, listOfMetricId).pipe( + map(metrics => { + let minValue = NaN; + const watermarks: { [index: string]: number } = {}; + for (const key in metrics.values) { + const value = metrics.values[key]; + const subtaskIndex = key.replace('.currentInputWatermark', ''); + watermarks[subtaskIndex] = value; + if (isNaN(minValue) || value < minValue) { + minValue = value; + } + } + if (!isNaN(minValue) && minValue > LONG_MIN_VALUE) { + lowWatermark = minValue; + } else { + lowWatermark = NaN; + } + return { ...node, lowWatermark }; + }) + ); + }) + ).pipe(catchError(() => of(nodes))); + } + + refreshNodesWithWatermarks() { + this.mergeWithWatermarks(this.nodes).subscribe(nodes => { + nodes.forEach(node => { + this.dagreComponent.updateNode(node.id, node); + }); + }); + } + constructor( private jobService: JobService, private router: Router, private activatedRoute: ActivatedRoute, public elementRef: ElementRef, + private metricService: MetricsService, private cdr: ChangeDetectorRef ) {} @@ -82,11 +124,10 @@ export class JobOverviewComponent implements OnInit, OnDestroy { this.links = data.plan.links; this.jobId = data.plan.jid; this.dagreComponent.flush(this.nodes, this.links, true).then(); + this.refreshNodesWithWatermarks(); } else { this.nodes = data.plan.nodes; - this.nodes.forEach(node => { - this.dagreComponent.updateNode(node.id, node); - }); + this.refreshNodesWithWatermarks(); } this.cdr.markForCheck(); }); diff --git a/flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.html b/flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.html index 799ad2b0dc..3b57b46bb8 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.html +++ b/flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.html @@ -23,6 +23,7 @@ {{operator}} {{description}} Parallelism: {{parallelism}} + Low Watermark {{lowWatermark}} Operation: {{operatorStrategy}} diff --git a/flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.less b/flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.less index 0f10e2d7da..0533e053ac 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.less +++ b/flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.less @@ -50,12 +50,17 @@ &:first-child { margin-top: 24px; } + + &.watermark { + font-weight: normal;; + font-weight: 12px; + color: @text-color-secondary; + } } .detail { margin-bottom: 12px; color: @text-color; - &.description { color: @heading-color; } diff --git a/flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.ts b/flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.ts index 2f96b12151..f24ef2411b 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/share/common/dagre/node.component.ts @@ -30,6 +30,7 @@ export class NodeComponent { operator: string | null; operatorStrategy: string | null; parallelism: number | null; + lowWatermark: number | null | undefined; height = 0; id: string; @@ -45,6 +46,7 @@ export class NodeComponent { this.operator = this.decodeHTML(value.operator); this.operatorStrategy = this.decodeHTML(value.operator_strategy); this.parallelism = value.parallelism; + this.lowWatermark = value.lowWatermark; this.height = value.height || 0; this.id = value.id; if (description && description.length > 300) { -- Gitee From 2a6eefacb05ba4d624f148ff57f4afbf3f5acf90 Mon Sep 17 00:00:00 2001 From: vthinkxie Date: Tue, 30 Jul 2019 10:42:34 +0800 Subject: [PATCH 071/268] [FLINK-13386][web]: Fix sort in Firefox --- .../pages/job/overview/list/job-overview-list.component.html | 4 ++-- .../pages/job/overview/list/job-overview-list.component.ts | 4 ++-- .../web-dashboard/src/app/services/job.service.ts | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/list/job-overview-list.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/list/job-overview-list.component.html index 8e2debe32a..f6d7399058 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/list/job-overview-list.component.html +++ b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/list/job-overview-list.component.html @@ -20,7 +20,7 @@ class="no-border small" [nzSize]="'small'" [nzData]="nodes" - [nzScroll]="{x:((1310+left)+'px')}" + [nzScroll]="{x:((1360+left)+'px')}" [nzFrontPagination]="false" [nzShowPagination]="false"> @@ -33,7 +33,7 @@ Records Sent Parallelism Start Time - Duration + Duration End Time Tasks diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/list/job-overview-list.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/list/job-overview-list.component.ts index e7cf66bfe6..c1ceaef583 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/list/job-overview-list.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/list/job-overview-list.component.ts @@ -28,8 +28,8 @@ import { NodesItemCorrectInterface } from 'interfaces'; }) export class JobOverviewListComponent { innerNodes: NodesItemCorrectInterface[] = []; - sortName = 'detail.topology-id'; - sortValue = 'ascend'; + sortName: string; + sortValue: string; left = 390; @Output() nodeClick = new EventEmitter(); @Input() selectedNode: NodesItemCorrectInterface; diff --git a/flink-runtime-web/web-dashboard/src/app/services/job.service.ts b/flink-runtime-web/web-dashboard/src/app/services/job.service.ts index 48f32a1dcd..4abaa3f04d 100644 --- a/flink-runtime-web/web-dashboard/src/app/services/job.service.ts +++ b/flink-runtime-web/web-dashboard/src/app/services/job.service.ts @@ -265,7 +265,7 @@ export class JobService { } }); const listOfVerticesId = job.vertices.map(item => item.id); - nodes.sort((pre, next) => listOfVerticesId.indexOf(next.id) - listOfVerticesId.indexOf(pre.id)); + nodes.sort((pre, next) => listOfVerticesId.indexOf(pre.id) - listOfVerticesId.indexOf(next.id)); } return { ...job, -- Gitee From 3a49da62332d4cfaf0536a11fb7da8de11baa17e Mon Sep 17 00:00:00 2001 From: tison Date: Sat, 28 Sep 2019 02:36:57 +0800 Subject: [PATCH 072/268] [FLINK-13992][coordination] Refactor Optional parameter in InputGateWithMetrics#updateMetrics As consensus from community code style discussion, in InputGateWithMetrics#updateMetrics we can refactor to reduce the usage of Optional parameter. This closes #9684. --- .../flink/runtime/taskmanager/InputGateWithMetrics.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/InputGateWithMetrics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/InputGateWithMetrics.java index 5d2cfd95c4..26ba5de036 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/InputGateWithMetrics.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/InputGateWithMetrics.java @@ -67,12 +67,12 @@ public class InputGateWithMetrics extends InputGate { @Override public Optional getNext() throws IOException, InterruptedException { - return updateMetrics(inputGate.getNext()); + return inputGate.getNext().map(this::updateMetrics); } @Override public Optional pollNext() throws IOException, InterruptedException { - return updateMetrics(inputGate.pollNext()); + return inputGate.pollNext().map(this::updateMetrics); } @Override @@ -85,8 +85,8 @@ public class InputGateWithMetrics extends InputGate { inputGate.close(); } - private Optional updateMetrics(Optional bufferOrEvent) { - bufferOrEvent.ifPresent(b -> numBytesIn.inc(b.getSize())); + private BufferOrEvent updateMetrics(BufferOrEvent bufferOrEvent) { + numBytesIn.inc(bufferOrEvent.getSize()); return bufferOrEvent; } } -- Gitee From bf3587d7da1073eee44f3f8a13a02baac7290c3a Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Fri, 27 Sep 2019 09:36:09 +0800 Subject: [PATCH 073/268] [FLINK-13656][sql-parser] Bump sql parser Calcite dependency to 1.21.0 * Add ExtendedSqlRowTypeNameSpec to represent custom ROW type name * Add SqlMapTypeNameSpec to represent SQL MAP type name * Add ExtendedSqlBuiltinTypeNameSpec to represent extended builtin type like STRING and BYTES * Add ExtendedSqlCollectionTypeNameSpec to represent custom collection type name * Remove original temp SqlXXXType files * Remove temp FlinkSqlDataTypeSpec because CALCITE-3213 has been resolved * In FlinkDDLDataTypeTest, support parsing for user defined type, now if user does not register UDT, it would throws during sql node validation --- flink-table/flink-sql-parser/pom.xml | 5 +- .../src/main/codegen/data/Parser.tdd | 42 +-- .../src/main/codegen/includes/parserImpls.ftl | 325 +++++++----------- .../sql/ExtendedSqlRowTypeNameSpec.java} | 106 ++++-- .../sql/SqlMapTypeNameSpec.java} | 64 +++- .../sql/package-info.java} | 25 +- .../flink/sql/parser/ExtendedSqlNode.java | 4 +- .../sql/parser/FlinkSqlDataTypeSpec.java | 325 ------------------ .../flink/sql/parser/ddl/SqlCreateTable.java | 12 +- .../flink/sql/parser/ddl/SqlCreateView.java | 4 +- .../flink/sql/parser/ddl/SqlTableColumn.java | 8 +- .../flink/sql/parser/dml/RichSqlInsert.java | 4 +- ...ception.java => SqlValidateException.java} | 11 +- ...java => ExtendedSqlBasicTypeNameSpec.java} | 42 ++- .../ExtendedSqlCollectionTypeNameSpec.java | 129 +++++++ .../sql/parser/type/ExtendedSqlType.java | 42 --- .../flink/sql/parser/type/SqlArrayType.java | 49 --- .../flink/sql/parser/type/SqlStringType.java | 38 -- .../flink/sql/parser/type/SqlTimeType.java | 73 ---- .../sql/parser/type/SqlTimestampType.java | 73 ---- .../sql/parser/FlinkDDLDataTypeTest.java | 206 ++++++++--- .../sql/parser/FlinkSqlParserImplTest.java | 32 +- flink-table/pom.xml | 1 + 23 files changed, 649 insertions(+), 971 deletions(-) rename flink-table/flink-sql-parser/src/main/java/org/apache/{flink/sql/parser/type/SqlRowType.java => calcite/sql/ExtendedSqlRowTypeNameSpec.java} (41%) rename flink-table/flink-sql-parser/src/main/java/org/apache/{flink/sql/parser/type/SqlMapType.java => calcite/sql/SqlMapTypeNameSpec.java} (45%) rename flink-table/flink-sql-parser/src/main/java/org/apache/{flink/sql/parser/type/SqlBytesType.java => calcite/sql/package-info.java} (58%) delete mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/FlinkSqlDataTypeSpec.java rename flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/error/{SqlParseException.java => SqlValidateException.java} (79%) rename flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/{SqlMultisetType.java => ExtendedSqlBasicTypeNameSpec.java} (53%) create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlCollectionTypeNameSpec.java delete mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlType.java delete mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlArrayType.java delete mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlStringType.java delete mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlTimeType.java delete mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlTimestampType.java diff --git a/flink-table/flink-sql-parser/pom.xml b/flink-table/flink-sql-parser/pom.xml index a4c9b5dfa2..67dcf3bbdc 100644 --- a/flink-table/flink-sql-parser/pom.xml +++ b/flink-table/flink-sql-parser/pom.xml @@ -35,7 +35,6 @@ under the License. - 1.20.0 @@ -46,9 +45,9 @@ under the License. ${calcite.version} -/** -* Parse a nullable option, default to be nullable. -*/ -boolean NullableOpt() : -{ -} -{ - { return true; } -| - { return false; } -| - { return true; } -} - void TableColumn(TableCreationContext context) : { } @@ -69,8 +55,7 @@ void TableColumn2(List list) : } { name = SimpleIdentifier() - <#-- #FlinkDataType already takes care of the nullable attribute. --> - type = FlinkDataType() + type = ExtendedDataType() [ { String p = SqlParserUtil.parseString(token.image); comment = SqlLiteral.createCharString(p, getPos()); @@ -81,6 +66,35 @@ void TableColumn2(List list) : } } +/** +* Different with {@link #DataType()}, we support a [ NULL | NOT NULL ] suffix syntax for both the +* collection element data type and the data type itself. +* +*

See {@link #SqlDataTypeSpec} for the syntax details of {@link #DataType()}. +*/ +SqlDataTypeSpec ExtendedDataType() : +{ + SqlTypeNameSpec typeName; + final Span s; + boolean elementNullable = true; + boolean nullable = true; +} +{ + <#-- #DataType does not take care of the nullable attribute. --> + typeName = TypeName() { + s = span(); + } + ( + LOOKAHEAD(3) + elementNullable = NullableOptDefaultTrue() + typeName = ExtendedCollectionsTypeName(typeName, elementNullable) + )* + nullable = NullableOptDefaultTrue() + { + return new SqlDataTypeSpec(typeName, s.end(this)).withNullable(nullable); + } +} + SqlNodeList PrimaryKey() : { List pkList = new ArrayList(); @@ -375,210 +389,100 @@ SqlDrop SqlDropView(Span s, boolean replace) : } } -SqlIdentifier FlinkCollectionsTypeName() : -{ -} -{ - LOOKAHEAD(2) - { - return new SqlIdentifier(SqlTypeName.MULTISET.name(), getPos()); - } -| - { - return new SqlIdentifier(SqlTypeName.ARRAY.name(), getPos()); - } -} - -SqlIdentifier FlinkTypeName() : +/** +* A sql type name extended basic data type, it has a counterpart basic +* sql type name but always represents as a special alias compared with the standard name. +* +*

For example, STRING is synonym of VARCHAR(INT_MAX) +* and BYTES is synonym of VARBINARY(INT_MAX). +*/ +SqlTypeNameSpec ExtendedSqlBasicTypeName() : { - final SqlTypeName sqlTypeName; - final SqlIdentifier typeName; - final Span s = Span.of(); + final SqlTypeName typeName; + final String typeAlias; + int precision = -1; } { ( -<#-- additional types are included here --> -<#-- make custom data types in front of Calcite core data types --> -<#list parser.flinkDataTypeParserMethods as method> - <#if (method?index > 0)> - | - - LOOKAHEAD(2) - typeName = ${method} - - | - LOOKAHEAD(2) - sqlTypeName = SqlTypeName(s) { - typeName = new SqlIdentifier(sqlTypeName.name(), s.end(this)); + { + typeName = SqlTypeName.VARCHAR; + typeAlias = token.image; + precision = Integer.MAX_VALUE; } | - LOOKAHEAD(2) - typeName = FlinkCollectionsTypeName() - | - typeName = CompoundIdentifier() { - throw new ParseException("UDT in DDL is not supported yet."); + { + typeName = SqlTypeName.VARBINARY; + typeAlias = token.image; + precision = Integer.MAX_VALUE; } ) { - return typeName; + return new ExtendedSqlBasicTypeNameSpec(typeAlias, typeName, precision, getPos()); } } -/** -* Parse a Flink data type with nullable options, NULL -> nullable, NOT NULL -> not nullable. -* Default to be nullable. +/* +* Parses collection type name that does not belong to standard SQL, i.e. ARRAY<INT NOT NULL>. */ -SqlDataTypeSpec FlinkDataType() : +SqlTypeNameSpec CustomizedCollectionsTypeName() : { - final SqlIdentifier typeName; - SqlIdentifier collectionTypeName = null; - int scale = -1; - int precision = -1; - String charSetName = null; - final Span s; - boolean nullable = true; + final SqlTypeName collectionTypeName; + final SqlTypeNameSpec elementTypeName; boolean elementNullable = true; } { - typeName = FlinkTypeName() { - s = span(); - } - [ - - precision = UnsignedIntLiteral() - [ - - scale = UnsignedIntLiteral() - ] - - ] - elementNullable = NullableOpt() - [ - collectionTypeName = FlinkCollectionsTypeName() - nullable = NullableOpt() - ] - { - if (null != collectionTypeName) { - return new FlinkSqlDataTypeSpec( - collectionTypeName, - typeName, - precision, - scale, - charSetName, - nullable, - elementNullable, - s.end(collectionTypeName)); - } - nullable = elementNullable; - return new FlinkSqlDataTypeSpec(typeName, - precision, - scale, - charSetName, - null, - nullable, - elementNullable, - s.end(this)); - } -} - -SqlIdentifier SqlStringType() : -{ -} -{ - { return new SqlStringType(getPos()); } -} - -SqlIdentifier SqlBytesType() : -{ -} -{ - { return new SqlBytesType(getPos()); } -} - -boolean WithLocalTimeZone() : -{ -} -{ -

The difference with {@link #SqlRowTypeName()}: +*

    +*
  • Support comment syntax for every field
  • +*
  • Field data type default is nullable
  • +*
  • Support ROW type with empty fields, e.g. ROW()
  • +*
*/ -SqlIdentifier SqlRowType() : +SqlTypeNameSpec ExtendedSqlRowTypeName() : { List fieldNames = new ArrayList(); List fieldTypes = new ArrayList(); List comments = new ArrayList(); + final boolean unparseAsStandard; } { ( - + { unparseAsStandard = false; } | - FieldNameTypeCommaList(fieldNames, fieldTypes, comments) + ExtendedFieldNameTypeCommaList(fieldNames, fieldTypes, comments) + { unparseAsStandard = false; } | - FieldNameTypeCommaList(fieldNames, fieldTypes, comments) + ExtendedFieldNameTypeCommaList(fieldNames, fieldTypes, comments) + { unparseAsStandard = true; } ) { - return new SqlRowType(getPos(), fieldNames, fieldTypes, comments); + return new ExtendedSqlRowTypeNameSpec( + getPos(), + fieldNames, + fieldTypes, + comments, + unparseAsStandard); } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlRowType.java b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/ExtendedSqlRowTypeNameSpec.java similarity index 41% rename from flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlRowType.java rename to flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/ExtendedSqlRowTypeNameSpec.java index 886125cd07..42828d8c8c 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlRowType.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/ExtendedSqlRowTypeNameSpec.java @@ -16,35 +16,56 @@ * limitations under the License. */ -package org.apache.flink.sql.parser.type; +package org.apache.calcite.sql; -import org.apache.calcite.sql.SqlCharStringLiteral; -import org.apache.calcite.sql.SqlDataTypeSpec; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.util.Litmus; import org.apache.calcite.util.Pair; import java.util.List; +import java.util.stream.Collectors; /** - * Parse column of Row type. + * A sql type name specification of ROW type. + * + *

The difference with {@link SqlRowTypeNameSpec}: + *

    + *
  • Support comment syntax for every field
  • + *
  • Field data type default is nullable
  • + *
  • Support ROW type with empty fields, e.g. ROW()
  • + *
*/ -public class SqlRowType extends SqlIdentifier implements ExtendedSqlType { +public class ExtendedSqlRowTypeNameSpec extends SqlTypeNameSpec { private final List fieldNames; private final List fieldTypes; private final List comments; - public SqlRowType(SqlParserPos pos, + private final boolean unparseAsStandard; + + /** + * Creates a ROW type specification. + * + * @param pos parser position + * @param fieldNames field names + * @param fieldTypes field data types + * @param comments field comments + * @param unparseAsStandard whether to unparse as standard SQL style + */ + public ExtendedSqlRowTypeNameSpec(SqlParserPos pos, List fieldNames, List fieldTypes, - List comments) { - super(SqlTypeName.ROW.getName(), pos); + List comments, + boolean unparseAsStandard) { + super(new SqlIdentifier(SqlTypeName.ROW.getName(), pos), pos); this.fieldNames = fieldNames; this.fieldTypes = fieldTypes; this.comments = comments; + this.unparseAsStandard = unparseAsStandard; } public List getFieldNames() { @@ -59,30 +80,30 @@ public class SqlRowType extends SqlIdentifier implements ExtendedSqlType { return comments; } - public int getArity() { - return fieldNames.size(); - } - - public SqlIdentifier getFieldName(int i) { - return fieldNames.get(i); - } - - public SqlDataTypeSpec getFieldType(int i) { - return fieldTypes.get(i); - } - @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { writer.print("ROW"); if (getFieldNames().size() == 0) { - writer.print("<>"); + if (unparseAsStandard) { + writer.print("()"); + } else { + writer.print("<>"); + } } else { - SqlWriter.Frame frame = writer.startList(SqlWriter.FrameTypeEnum.FUN_CALL, "<", ">"); + SqlWriter.Frame frame; + if (unparseAsStandard) { + frame = writer.startList(SqlWriter.FrameTypeEnum.FUN_CALL, "(", ")"); + } else { + frame = writer.startList(SqlWriter.FrameTypeEnum.FUN_CALL, "<", ">"); + } int i = 0; for (Pair p : Pair.zip(this.fieldNames, this.fieldTypes)) { writer.sep(",", false); p.left.unparse(writer, 0, 0); - ExtendedSqlType.unparseType(p.right, writer, leftPrec, rightPrec); + p.right.unparse(writer, leftPrec, rightPrec); + if (!p.right.getNullable()) { + writer.keyword("NOT NULL"); + } if (comments.get(i) != null) { comments.get(i).unparse(writer, leftPrec, rightPrec); } @@ -91,4 +112,39 @@ public class SqlRowType extends SqlIdentifier implements ExtendedSqlType { writer.endList(frame); } } + + @Override public boolean equalsDeep(SqlTypeNameSpec node, Litmus litmus) { + if (!(node instanceof SqlRowTypeNameSpec)) { + return litmus.fail("{} != {}", this, node); + } + ExtendedSqlRowTypeNameSpec that = (ExtendedSqlRowTypeNameSpec) node; + if (this.fieldNames.size() != that.fieldNames.size()) { + return litmus.fail("{} != {}", this, node); + } + for (int i = 0; i < fieldNames.size(); i++) { + if (!this.fieldNames.get(i).equalsDeep(that.fieldNames.get(i), litmus)) { + return litmus.fail("{} != {}", this, node); + } + } + if (this.fieldTypes.size() != that.fieldTypes.size()) { + return litmus.fail("{} != {}", this, node); + } + for (int i = 0; i < fieldTypes.size(); i++) { + if (!this.fieldTypes.get(i).equals(that.fieldTypes.get(i))) { + return litmus.fail("{} != {}", this, node); + } + } + return litmus.succeed(); + } + + @Override public RelDataType deriveType(SqlValidator sqlValidator) { + final RelDataTypeFactory typeFactory = sqlValidator.getTypeFactory(); + return typeFactory.createStructType( + fieldTypes.stream() + .map(dt -> dt.deriveType(sqlValidator)) + .collect(Collectors.toList()), + fieldNames.stream() + .map(SqlIdentifier::toString) + .collect(Collectors.toList())); + } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlMapType.java b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlMapTypeNameSpec.java similarity index 45% rename from flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlMapType.java rename to flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlMapTypeNameSpec.java index fd071c09c8..0e22dbfd6d 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlMapType.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlMapTypeNameSpec.java @@ -16,24 +16,37 @@ * limitations under the License. */ -package org.apache.flink.sql.parser.type; +package org.apache.calcite.sql; -import org.apache.calcite.sql.SqlDataTypeSpec; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.util.Litmus; /** - * Parse column of Map type. + * Parse SQL MAP type, i.e. MAP<INT NOT NULL, TIMESTAMP NULL>, the key and value can specify + * a suffix to indicate if the type is nullable, default is nullable. + * + *

MAP type does not belong to standard SQL. */ -public class SqlMapType extends SqlIdentifier implements ExtendedSqlType { +public class SqlMapTypeNameSpec extends SqlTypeNameSpec { private final SqlDataTypeSpec keyType; private final SqlDataTypeSpec valType; - public SqlMapType(SqlParserPos pos, SqlDataTypeSpec keyType, SqlDataTypeSpec valType) { - super(SqlTypeName.MAP.getName(), pos); + /** + * Creates a {@code SqlMapTypeNameSpec}. + * + * @param keyType key type + * @param valType value type + * @param pos the parser position + */ + public SqlMapTypeNameSpec( + SqlDataTypeSpec keyType, + SqlDataTypeSpec valType, + SqlParserPos pos) { + super(new SqlIdentifier(SqlTypeName.MAP.getName(), pos), pos); this.keyType = keyType; this.valType = valType; } @@ -46,14 +59,45 @@ public class SqlMapType extends SqlIdentifier implements ExtendedSqlType { return valType; } + @Override + public RelDataType deriveType(SqlValidator validator) { + return validator.getTypeFactory() + .createMapType( + keyType.deriveType(validator), + valType.deriveType(validator)); + } + @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { writer.keyword("MAP"); SqlWriter.Frame frame = writer.startList(SqlWriter.FrameTypeEnum.FUN_CALL, "<", ">"); writer.sep(","); - ExtendedSqlType.unparseType(keyType, writer, leftPrec, rightPrec); + keyType.unparse(writer, leftPrec, rightPrec); + // Default is nullable. + if (!keyType.getNullable()) { + writer.keyword("NOT NULL"); + } writer.sep(","); - ExtendedSqlType.unparseType(valType, writer, leftPrec, rightPrec); + valType.unparse(writer, leftPrec, rightPrec); + // Default is nullable. + if (!valType.getNullable()) { + writer.keyword("NOT NULL"); + } writer.endList(frame); } + + @Override + public boolean equalsDeep(SqlTypeNameSpec spec, Litmus litmus) { + if (!(spec instanceof SqlMapTypeNameSpec)) { + return litmus.fail("{} != {}", this, spec); + } + SqlMapTypeNameSpec that = (SqlMapTypeNameSpec) spec; + if (!this.keyType.equalsDeep(that.keyType, litmus)) { + return litmus.fail("{} != {}", this, spec); + } + if (!this.valType.equalsDeep(that.valType, litmus)) { + return litmus.fail("{} != {}", this, spec); + } + return litmus.succeed(); + } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlBytesType.java b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/package-info.java similarity index 58% rename from flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlBytesType.java rename to flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/package-info.java index dfc2d1fdee..439912b52b 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlBytesType.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/package-info.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,23 +16,12 @@ * limitations under the License. */ -package org.apache.flink.sql.parser.type; - -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlWriter; -import org.apache.calcite.sql.parser.SqlParserPos; - /** - * Parse type "BYTES" which is a synonym of VARBINARY(INT_MAX). + * This package is needed because the constructor of SqlTypeNameSpec is package scope, + * we should merge this package into org.apache.flink.sql.parser.type when CALCITE-3360 + * is resolved. */ -public class SqlBytesType extends SqlIdentifier implements ExtendedSqlType { - - public SqlBytesType(SqlParserPos pos) { - super("BYTES", pos); - } +@PackageMarker +package org.apache.calcite.sql; - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - writer.keyword("BYTES"); - } -} +import org.apache.calcite.avatica.util.PackageMarker; diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ExtendedSqlNode.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ExtendedSqlNode.java index df541922b0..5c5c69be47 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ExtendedSqlNode.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ExtendedSqlNode.java @@ -18,7 +18,7 @@ package org.apache.flink.sql.parser; -import org.apache.flink.sql.parser.error.SqlParseException; +import org.apache.flink.sql.parser.error.SqlValidateException; /** An remark interface which should be inherited by extended sql nodes which are not supported * by Calcite core parser. @@ -27,5 +27,5 @@ import org.apache.flink.sql.parser.error.SqlParseException; * {@link org.apache.calcite.sql.validate.SqlValidatorImpl}. */ public interface ExtendedSqlNode { - void validate() throws SqlParseException; + void validate() throws SqlValidateException; } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/FlinkSqlDataTypeSpec.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/FlinkSqlDataTypeSpec.java deleted file mode 100644 index a3797d7a5d..0000000000 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/FlinkSqlDataTypeSpec.java +++ /dev/null @@ -1,325 +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.sql.parser; - -import org.apache.flink.sql.parser.type.ExtendedSqlType; -import org.apache.flink.sql.parser.type.SqlArrayType; -import org.apache.flink.sql.parser.type.SqlBytesType; -import org.apache.flink.sql.parser.type.SqlMapType; -import org.apache.flink.sql.parser.type.SqlMultisetType; -import org.apache.flink.sql.parser.type.SqlRowType; -import org.apache.flink.sql.parser.type.SqlStringType; -import org.apache.flink.sql.parser.type.SqlTimeType; -import org.apache.flink.sql.parser.type.SqlTimestampType; - -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.calcite.sql.SqlCollation; -import org.apache.calcite.sql.SqlDataTypeSpec; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlUtil; -import org.apache.calcite.sql.SqlWriter; -import org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.calcite.sql.type.SqlTypeUtil; -import org.apache.calcite.util.Util; - -import java.nio.charset.Charset; -import java.util.Objects; -import java.util.TimeZone; -import java.util.stream.Collectors; - -/** - * Represents a SQL data type specification in a parse tree. - * - *

A SqlDataTypeSpec is immutable; once created, you cannot - * change any of the fields.

- * - *

This class is an extension to {@link SqlDataTypeSpec}, we support - * complex type expressions like:

- * - *
ROW(
- * foo NUMBER(5, 2) NOT NULL,
- * rec ROW(b BOOLEAN, i MyUDT NOT NULL))
- * - *

Until CALCITE-3213 - * is resolved, we can remove this class. - */ -public class FlinkSqlDataTypeSpec extends SqlDataTypeSpec { - // Flag saying if the element type is nullable if this type is a collection type. - // For collection type, we mean ARRAY and MULTISET type now. - private Boolean elementNullable; - - public FlinkSqlDataTypeSpec( - SqlIdentifier collectionsTypeName, - SqlIdentifier typeName, - int precision, - int scale, - String charSetName, - Boolean nullable, - Boolean elementNullable, - SqlParserPos pos) { - super(collectionsTypeName, typeName, precision, scale, - charSetName, null, nullable, pos); - this.elementNullable = elementNullable; - } - - public FlinkSqlDataTypeSpec( - SqlIdentifier collectionsTypeName, - SqlIdentifier typeName, - int precision, - int scale, - String charSetName, - TimeZone timeZone, - Boolean nullable, - Boolean elementNullable, - SqlParserPos pos) { - super(collectionsTypeName, typeName, precision, scale, - charSetName, timeZone, nullable, pos); - this.elementNullable = elementNullable; - } - - public FlinkSqlDataTypeSpec( - SqlIdentifier typeName, - int precision, - int scale, - String charSetName, - TimeZone timeZone, - Boolean nullable, - Boolean elementNullable, - SqlParserPos pos) { - super(null, typeName, precision, scale, - charSetName, timeZone, nullable, pos); - this.elementNullable = elementNullable; - } - - @Override - public SqlNode clone(SqlParserPos pos) { - return (getCollectionsTypeName() != null) - ? new FlinkSqlDataTypeSpec(getCollectionsTypeName(), getTypeName(), getPrecision(), - getScale(), getCharSetName(), getNullable(), this.elementNullable, pos) - : new FlinkSqlDataTypeSpec(getTypeName(), getPrecision(), getScale(), - getCharSetName(), getTimeZone(), getNullable(), this.elementNullable, pos); - } - - /** Returns a copy of this data type specification with a given - * nullability. */ - @Override - public SqlDataTypeSpec withNullable(Boolean nullable) { - if (Objects.equals(nullable, this.getNullable())) { - return this; - } - return new FlinkSqlDataTypeSpec(getCollectionsTypeName(), getTypeName(), - getPrecision(), getScale(), getCharSetName(), getTimeZone(), nullable, - this.elementNullable, getParserPosition()); - } - - @Override - public RelDataType deriveType(RelDataTypeFactory typeFactory) { - // Default to be nullable. - return this.deriveType(typeFactory, true); - } - - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - final SqlIdentifier typeName = getTypeName(); - String name = typeName.getSimple(); - if (typeName instanceof ExtendedSqlType) { - typeName.unparse(writer, leftPrec, rightPrec); - } else if (SqlTypeName.get(name) != null) { - SqlTypeName sqlTypeName = SqlTypeName.get(name); - writer.keyword(name); - if (sqlTypeName.allowsPrec() && this.getPrecision() >= 0) { - SqlWriter.Frame frame = writer.startList(SqlWriter.FrameTypeEnum.FUN_CALL, "(", ")"); - writer.print(this.getPrecision()); - if (sqlTypeName.allowsScale() && this.getScale() >= 0) { - writer.sep(",", true); - writer.print(this.getScale()); - } - - writer.endList(frame); - } - - if (this.getCharSetName() != null) { - writer.keyword("CHARACTER SET"); - writer.identifier(this.getCharSetName(), false); - } - - if (this.getCollectionsTypeName() != null) { - // Fix up nullable attribute if this is a collection type. - if (elementNullable != null && !elementNullable) { - writer.keyword("NOT NULL"); - } - writer.keyword(this.getCollectionsTypeName().getSimple()); - } - } else if (name.startsWith("_")) { - writer.keyword(name.substring(1)); - } else { - this.getTypeName().unparse(writer, leftPrec, rightPrec); - } - if (getNullable() != null && !getNullable()) { - writer.keyword("NOT NULL"); - } - } - - @Override - public RelDataType deriveType(RelDataTypeFactory typeFactory, boolean nullable) { - final SqlIdentifier typeName = getTypeName(); - if (!typeName.isSimple()) { - return null; - } - final String name = typeName.getSimple(); - final SqlTypeName sqlTypeName = SqlTypeName.get(name); - // Try to get Flink custom data type first. - RelDataType type = getExtendedType(typeFactory, typeName); - if (type == null) { - if (sqlTypeName == null) { - return null; - } else { - // NOTE jvs 15-Jan-2009: earlier validation is supposed to - // have caught these, which is why it's OK for them - // to be assertions rather than user-level exceptions. - final int precision = getPrecision(); - final int scale = getScale(); - if ((precision >= 0) && (scale >= 0)) { - assert sqlTypeName.allowsPrecScale(true, true); - type = typeFactory.createSqlType(sqlTypeName, precision, scale); - } else if (precision >= 0) { - assert sqlTypeName.allowsPrecNoScale(); - type = typeFactory.createSqlType(sqlTypeName, precision); - } else { - assert sqlTypeName.allowsNoPrecNoScale(); - type = typeFactory.createSqlType(sqlTypeName); - } - } - } - - if (SqlTypeUtil.inCharFamily(type)) { - // Applying Syntax rule 10 from SQL:99 spec section 6.22 "If TD is a - // fixed-length, variable-length or large object character string, - // then the collating sequence of the result of the is the default collating sequence for the - // character repertoire of TD and the result of the has the Coercible coercibility characteristic." - SqlCollation collation = SqlCollation.COERCIBLE; - - Charset charset; - final String charSetName = getCharSetName(); - if (null == charSetName) { - charset = typeFactory.getDefaultCharset(); - } else { - String javaCharSetName = - Objects.requireNonNull( - SqlUtil.translateCharacterSetName(charSetName), charSetName); - charset = Charset.forName(javaCharSetName); - } - type = - typeFactory.createTypeWithCharsetAndCollation( - type, - charset, - collation); - } - - final SqlIdentifier collectionsTypeName = getCollectionsTypeName(); - if (null != collectionsTypeName) { - // Fix the nullability of the element type first. - boolean elementNullable = true; - if (this.elementNullable != null) { - elementNullable = this.elementNullable; - } - type = typeFactory.createTypeWithNullability(type, elementNullable); - - final String collectionName = collectionsTypeName.getSimple(); - final SqlTypeName collectionsSqlTypeName = - Objects.requireNonNull(SqlTypeName.get(collectionName), - collectionName); - - switch (collectionsSqlTypeName) { - case MULTISET: - type = typeFactory.createMultisetType(type, -1); - break; - case ARRAY: - type = typeFactory.createArrayType(type, -1); - break; - default: - throw Util.unexpected(collectionsSqlTypeName); - } - } - - // Fix the nullability of this type. - if (this.getNullable() != null) { - nullable = this.getNullable(); - } - type = typeFactory.createTypeWithNullability(type, nullable); - - return type; - } - - private RelDataType getExtendedType(RelDataTypeFactory typeFactory, SqlIdentifier typeName) { - // quick check. - if (!(typeName instanceof ExtendedSqlType)) { - return null; - } - if (typeName instanceof SqlBytesType) { - return typeFactory.createSqlType(SqlTypeName.VARBINARY, Integer.MAX_VALUE); - } else if (typeName instanceof SqlStringType) { - return typeFactory.createSqlType(SqlTypeName.VARCHAR, Integer.MAX_VALUE); - } else if (typeName instanceof SqlArrayType) { - final SqlArrayType arrayType = (SqlArrayType) typeName; - return typeFactory.createArrayType(arrayType.getElementType() - .deriveType(typeFactory), -1); - } else if (typeName instanceof SqlMultisetType) { - final SqlMultisetType multiSetType = (SqlMultisetType) typeName; - return typeFactory.createMultisetType(multiSetType.getElementType() - .deriveType(typeFactory), -1); - } else if (typeName instanceof SqlMapType) { - final SqlMapType mapType = (SqlMapType) typeName; - return typeFactory.createMapType( - mapType.getKeyType().deriveType(typeFactory), - mapType.getValType().deriveType(typeFactory)); - } else if (typeName instanceof SqlRowType) { - final SqlRowType rowType = (SqlRowType) typeName; - return typeFactory.createStructType( - rowType.getFieldTypes().stream().map(ft -> ft.deriveType(typeFactory)) - .collect(Collectors.toList()), - rowType.getFieldNames().stream().map(SqlIdentifier::getSimple) - .collect(Collectors.toList())); - } else if (typeName instanceof SqlTimeType) { - final SqlTimeType zonedTimeType = (SqlTimeType) typeName; - if (zonedTimeType.getPrecision() >= 0) { - return typeFactory.createSqlType(zonedTimeType.getSqlTypeName(), - zonedTimeType.getPrecision()); - } else { - // Use default precision. - return typeFactory.createSqlType(zonedTimeType.getSqlTypeName()); - } - } else if (typeName instanceof SqlTimestampType) { - final SqlTimestampType zonedTimestampType = (SqlTimestampType) typeName; - if (zonedTimestampType.getPrecision() >= 0) { - return typeFactory.createSqlType(zonedTimestampType.getSqlTypeName(), - zonedTimestampType.getPrecision()); - } else { - // Use default precision. - return typeFactory.createSqlType(zonedTimestampType.getSqlTypeName()); - } - } - return null; - } -} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java index 183e2d8e7e..4fdd43370f 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java @@ -19,7 +19,7 @@ package org.apache.flink.sql.parser.ddl; import org.apache.flink.sql.parser.ExtendedSqlNode; -import org.apache.flink.sql.parser.error.SqlParseException; +import org.apache.flink.sql.parser.error.SqlValidateException; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlCall; @@ -127,7 +127,7 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { return ifNotExists; } - public void validate() throws SqlParseException { + public void validate() throws SqlValidateException { Set columnNames = new HashSet<>(); if (columnList != null) { for (SqlNode column : columnList) { @@ -141,7 +141,7 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { } if (!columnNames.add(columnName)) { - throw new SqlParseException( + throw new SqlValidateException( column.getParserPosition(), "Duplicate column name [" + columnName + "], at " + column.getParserPosition()); @@ -153,7 +153,7 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { for (SqlNode primaryKeyNode : this.primaryKeyList) { String primaryKey = ((SqlIdentifier) primaryKeyNode).getSimple(); if (!columnNames.contains(primaryKey)) { - throw new SqlParseException( + throw new SqlValidateException( primaryKeyNode.getParserPosition(), "Primary key [" + primaryKey + "] not defined in columns, at " + primaryKeyNode.getParserPosition()); @@ -166,7 +166,7 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { for (SqlNode uniqueKeyNode : uniqueKeys) { String uniqueKey = ((SqlIdentifier) uniqueKeyNode).getSimple(); if (!columnNames.contains(uniqueKey)) { - throw new SqlParseException( + throw new SqlValidateException( uniqueKeyNode.getParserPosition(), "Unique key [" + uniqueKey + "] not defined in columns, at " + uniqueKeyNode.getParserPosition()); } @@ -178,7 +178,7 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { for (SqlNode partitionKeyNode : this.partitionKeyList.getList()) { String partitionKey = ((SqlIdentifier) partitionKeyNode).getSimple(); if (!columnNames.contains(partitionKey)) { - throw new SqlParseException( + throw new SqlValidateException( partitionKeyNode.getParserPosition(), "Partition column [" + partitionKey + "] not defined in columns, at " + partitionKeyNode.getParserPosition()); diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateView.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateView.java index b77af04832..0bb193b31c 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateView.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateView.java @@ -19,7 +19,7 @@ package org.apache.flink.sql.parser.ddl; import org.apache.flink.sql.parser.ExtendedSqlNode; -import org.apache.flink.sql.parser.error.SqlParseException; +import org.apache.flink.sql.parser.error.SqlValidateException; import org.apache.calcite.sql.SqlCharStringLiteral; import org.apache.calcite.sql.SqlCreate; @@ -115,7 +115,7 @@ public class SqlCreateView extends SqlCreate implements ExtendedSqlNode { } @Override - public void validate() throws SqlParseException { + public void validate() throws SqlValidateException { // no-op } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlTableColumn.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlTableColumn.java index db9594be40..8a63683e27 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlTableColumn.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlTableColumn.java @@ -18,8 +18,6 @@ package org.apache.flink.sql.parser.ddl; -import org.apache.flink.sql.parser.type.ExtendedSqlType; - import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlCharStringLiteral; import org.apache.calcite.sql.SqlDataTypeSpec; @@ -71,7 +69,11 @@ public class SqlTableColumn extends SqlCall { public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { this.name.unparse(writer, leftPrec, rightPrec); writer.print(" "); - ExtendedSqlType.unparseType(type, writer, leftPrec, rightPrec); + this.type.unparse(writer, leftPrec, rightPrec); + if (!this.type.getNullable()) { + // Default is nullable. + writer.keyword("NOT NULL"); + } if (this.comment != null) { writer.print(" COMMENT "); this.comment.unparse(writer, leftPrec, rightPrec); diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dml/RichSqlInsert.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dml/RichSqlInsert.java index 4b681b2955..5d7a169457 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dml/RichSqlInsert.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dml/RichSqlInsert.java @@ -20,7 +20,7 @@ package org.apache.flink.sql.parser.dml; import org.apache.flink.sql.parser.ExtendedSqlNode; import org.apache.flink.sql.parser.SqlProperty; -import org.apache.flink.sql.parser.error.SqlParseException; +import org.apache.flink.sql.parser.error.SqlValidateException; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlInsertKeyword; @@ -142,7 +142,7 @@ public class RichSqlInsert extends SqlInsert implements ExtendedSqlNode { } @Override - public void validate() throws SqlParseException { + public void validate() throws SqlValidateException { // no-op } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/error/SqlParseException.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/error/SqlValidateException.java similarity index 79% rename from flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/error/SqlParseException.java rename to flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/error/SqlValidateException.java index 365de20e72..9ee46d160b 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/error/SqlParseException.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/error/SqlValidateException.java @@ -21,22 +21,21 @@ package org.apache.flink.sql.parser.error; import org.apache.calcite.sql.parser.SqlParserPos; /** - * SQL parse Exception. This is a simpler version - * of Calcite {@link org.apache.calcite.sql.parser.SqlParseException} - * which is used for SqlNode validation. + * SQL parse Exception. This exception mainly throws during + * {@link org.apache.flink.sql.parser.ExtendedSqlNode} validation. */ -public class SqlParseException extends Exception { +public class SqlValidateException extends Exception { private SqlParserPos errorPosition; private String message; - public SqlParseException(SqlParserPos errorPosition, String message) { + public SqlValidateException(SqlParserPos errorPosition, String message) { this.errorPosition = errorPosition; this.message = message; } - public SqlParseException(SqlParserPos errorPosition, String message, Exception e) { + public SqlValidateException(SqlParserPos errorPosition, String message, Exception e) { super(e); this.errorPosition = errorPosition; this.message = message; diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlMultisetType.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlBasicTypeNameSpec.java similarity index 53% rename from flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlMultisetType.java rename to flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlBasicTypeNameSpec.java index 3b6f19c2e6..ec81a0dd6c 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlMultisetType.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlBasicTypeNameSpec.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -18,32 +18,40 @@ package org.apache.flink.sql.parser.type; -import org.apache.calcite.sql.SqlDataTypeSpec; -import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlBasicTypeNameSpec; import org.apache.calcite.sql.SqlWriter; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlTypeName; /** - * Parse column of MULTISET type. + * A sql type name specification of extended basic data type, it has a counterpart basic + * sql type name but always represents as a special alias in Flink. + * + *

For example, STRING is synonym of VARCHAR(INT_MAX) + * and BYTES is synonym of VARBINARY(INT_MAX). */ -public class SqlMultisetType extends SqlIdentifier implements ExtendedSqlType { - - private final SqlDataTypeSpec elementType; - - public SqlMultisetType(SqlParserPos pos, SqlDataTypeSpec elementType) { - super(SqlTypeName.MULTISET.getName(), pos); - this.elementType = elementType; - } +public class ExtendedSqlBasicTypeNameSpec extends SqlBasicTypeNameSpec { + // Type alias used for unparsing. + private final String typeAlias; - public SqlDataTypeSpec getElementType() { - return elementType; + /** + * Creates a {@code ExtendedSqlBuiltinTypeNameSpec} instance. + * + * @param typeName type name + * @param precision type precision + * @param pos parser position + */ + public ExtendedSqlBasicTypeNameSpec( + String typeAlias, + SqlTypeName typeName, + int precision, + SqlParserPos pos) { + super(typeName, precision, pos); + this.typeAlias = typeAlias; } @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - writer.keyword("MULTISET<"); - ExtendedSqlType.unparseType(this.elementType, writer, leftPrec, rightPrec); - writer.keyword(">"); + writer.keyword(typeAlias); } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlCollectionTypeNameSpec.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlCollectionTypeNameSpec.java new file mode 100644 index 0000000000..a69039231e --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlCollectionTypeNameSpec.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.sql.parser.type; + +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.sql.SqlCollectionTypeNameSpec; +import org.apache.calcite.sql.SqlTypeNameSpec; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.util.Litmus; +import org.apache.calcite.util.Util; + +/** + * A extended sql type name specification of collection type, + * different with {@link SqlCollectionTypeNameSpec}, + * we support NULL or NOT NULL suffix for the element type name(this syntax + * does not belong to standard SQL). + */ +public class ExtendedSqlCollectionTypeNameSpec extends SqlCollectionTypeNameSpec { + private final boolean elementNullable; + private final SqlTypeName collectionTypeName; + private final boolean unparseAsStandard; + + /** + * Creates a {@code ExtendedSqlCollectionTypeNameSpec}. + * + * @param elementTypeName element type name specification + * @param elementNullable flag indicating if the element type is nullable + * @param collectionTypeName collection type name + * @param unparseAsStandard if we should unparse the collection type as standard SQL + * style + * @param pos the parser position + */ + public ExtendedSqlCollectionTypeNameSpec( + SqlTypeNameSpec elementTypeName, + boolean elementNullable, + SqlTypeName collectionTypeName, + boolean unparseAsStandard, + SqlParserPos pos) { + super(elementTypeName, collectionTypeName, pos); + this.elementNullable = elementNullable; + this.collectionTypeName = collectionTypeName; + this.unparseAsStandard = unparseAsStandard; + } + + @Override + public RelDataType deriveType(SqlValidator validator) { + RelDataType elementType = getElementTypeName().deriveType(validator); + elementType = validator.getTypeFactory() + .createTypeWithNullability(elementType, elementNullable); + return createCollectionType(elementType, validator.getTypeFactory()); + } + + @Override + public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { + if (unparseAsStandard) { + this.getElementTypeName().unparse(writer, leftPrec, rightPrec); + // Default is nullable. + if (!elementNullable) { + writer.keyword("NOT NULL"); + } + writer.keyword(collectionTypeName.name()); + } else { + writer.keyword(collectionTypeName.name()); + SqlWriter.Frame frame = writer.startList(SqlWriter.FrameTypeEnum.FUN_CALL, "<", ">"); + + getElementTypeName().unparse(writer, leftPrec, rightPrec); + // Default is nullable. + if (!elementNullable) { + writer.keyword("NOT NULL"); + } + writer.endList(frame); + } + } + + @Override + public boolean equalsDeep(SqlTypeNameSpec spec, Litmus litmus) { + if (!(spec instanceof ExtendedSqlCollectionTypeNameSpec)) { + return litmus.fail("{} != {}", this, spec); + } + ExtendedSqlCollectionTypeNameSpec that = (ExtendedSqlCollectionTypeNameSpec) spec; + if (this.elementNullable != that.elementNullable) { + return litmus.fail("{} != {}", this, spec); + } + return super.equalsDeep(spec, litmus); + } + + //~ Tools ------------------------------------------------------------------ + + /** + * Create collection data type. + * + * @param elementType Type of the collection element + * @param typeFactory Type factory + * @return The collection data type, or throw exception if the collection + * type name does not belong to {@code SqlTypeName} enumerations + */ + private RelDataType createCollectionType(RelDataType elementType, + RelDataTypeFactory typeFactory) { + switch (collectionTypeName) { + case MULTISET: + return typeFactory.createMultisetType(elementType, -1); + case ARRAY: + return typeFactory.createArrayType(elementType, -1); + + default: + throw Util.unexpected(collectionTypeName); + } + } +} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlType.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlType.java deleted file mode 100644 index f27656db62..0000000000 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlType.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.sql.parser.type; - -import org.apache.calcite.sql.SqlDataTypeSpec; -import org.apache.calcite.sql.SqlWriter; - -/** An remark interface which should be inherited by supported sql types which are not supported - * by Calcite default parser. - * - *

Caution that the subclass must override the method - * {@link org.apache.calcite.sql.SqlNode#unparse(SqlWriter, int, int)}. - */ -public interface ExtendedSqlType { - - static void unparseType(SqlDataTypeSpec type, - SqlWriter writer, - int leftPrec, - int rightPrec) { - if (type.getTypeName() instanceof ExtendedSqlType) { - type.getTypeName().unparse(writer, leftPrec, rightPrec); - } else { - type.unparse(writer, leftPrec, rightPrec); - } - } -} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlArrayType.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlArrayType.java deleted file mode 100644 index 336f883732..0000000000 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlArrayType.java +++ /dev/null @@ -1,49 +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.sql.parser.type; - -import org.apache.calcite.sql.SqlDataTypeSpec; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlWriter; -import org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.calcite.sql.type.SqlTypeName; - -/** - * Parse column of ArrayType. - */ -public class SqlArrayType extends SqlIdentifier implements ExtendedSqlType { - - private final SqlDataTypeSpec elementType; - - public SqlArrayType(SqlParserPos pos, SqlDataTypeSpec elementType) { - super(SqlTypeName.ARRAY.getName(), pos); - this.elementType = elementType; - } - - public SqlDataTypeSpec getElementType() { - return elementType; - } - - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - writer.keyword("ARRAY<"); - ExtendedSqlType.unparseType(this.elementType, writer, leftPrec, rightPrec); - writer.keyword(">"); - } -} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlStringType.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlStringType.java deleted file mode 100644 index a134b13abe..0000000000 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlStringType.java +++ /dev/null @@ -1,38 +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.sql.parser.type; - -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlWriter; -import org.apache.calcite.sql.parser.SqlParserPos; - -/** - * Parse type "STRING" which is a synonym of VARCHAR(INT_MAX). - */ -public class SqlStringType extends SqlIdentifier implements ExtendedSqlType { - - public SqlStringType(SqlParserPos pos) { - super("STRING", pos); - } - - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - writer.keyword("STRING"); - } -} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlTimeType.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlTimeType.java deleted file mode 100644 index 23855ce7af..0000000000 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlTimeType.java +++ /dev/null @@ -1,73 +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.sql.parser.type; - -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlWriter; -import org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.calcite.sql.type.SqlTypeName; - -/** - * Parse type "TIME WITHOUT TIME ZONE", "TIME(3) WITHOUT TIME ZONE", "TIME WITH LOCAL TIME ZONE", - * or "TIME(3) WITH LOCAL TIME ZONE". - */ -public class SqlTimeType extends SqlIdentifier implements ExtendedSqlType { - private final int precision; - private final boolean withLocalTimeZone; - - public SqlTimeType(SqlParserPos pos, int precision, boolean withLocalTimeZone) { - super(getTypeName(withLocalTimeZone), pos); - this.precision = precision; - this.withLocalTimeZone = withLocalTimeZone; - } - - private static String getTypeName(boolean withLocalTimeZone) { - if (withLocalTimeZone) { - return SqlTypeName.TIME_WITH_LOCAL_TIME_ZONE.name(); - } else { - return SqlTypeName.TIME.name(); - } - } - - public SqlTypeName getSqlTypeName() { - if (withLocalTimeZone) { - return SqlTypeName.TIME_WITH_LOCAL_TIME_ZONE; - } else { - return SqlTypeName.TIME; - } - } - - public int getPrecision() { - return this.precision; - } - - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - writer.keyword(SqlTypeName.TIME.name()); - if (this.precision >= 0) { - final SqlWriter.Frame frame = - writer.startList(SqlWriter.FrameTypeEnum.FUN_CALL, "(", ")"); - writer.print(precision); - writer.endList(frame); - } - if (this.withLocalTimeZone) { - writer.keyword("WITH LOCAL TIME ZONE"); - } - } -} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlTimestampType.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlTimestampType.java deleted file mode 100644 index 09e2d08bba..0000000000 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlTimestampType.java +++ /dev/null @@ -1,73 +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.sql.parser.type; - -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlWriter; -import org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.calcite.sql.type.SqlTypeName; - -/** - * Parse type "TIMESTAMP WITHOUT TIME ZONE", "TIMESTAMP(3) WITHOUT TIME ZONE", - * "TIMESTAMP WITH LOCAL TIME ZONE", or "TIMESTAMP(3) WITH LOCAL TIME ZONE". - */ -public class SqlTimestampType extends SqlIdentifier implements ExtendedSqlType { - private final int precision; - private final boolean withLocalTimeZone; - - public SqlTimestampType(SqlParserPos pos, int precision, boolean withLocalTimeZone) { - super(getTypeName(withLocalTimeZone), pos); - this.precision = precision; - this.withLocalTimeZone = withLocalTimeZone; - } - - private static String getTypeName(boolean withLocalTimeZone) { - if (withLocalTimeZone) { - return SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE.name(); - } else { - return SqlTypeName.TIMESTAMP.name(); - } - } - - public SqlTypeName getSqlTypeName() { - if (withLocalTimeZone) { - return SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE; - } else { - return SqlTypeName.TIMESTAMP; - } - } - - public int getPrecision() { - return this.precision; - } - - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - writer.keyword(SqlTypeName.TIMESTAMP.name()); - if (this.precision >= 0) { - final SqlWriter.Frame frame = - writer.startList(SqlWriter.FrameTypeEnum.FUN_CALL, "(", ")"); - writer.print(precision); - writer.endList(frame); - } - if (this.withLocalTimeZone) { - writer.keyword("WITH LOCAL TIME ZONE"); - } - } -} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkDDLDataTypeTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkDDLDataTypeTest.java index 410e30de6e..f5edc4f4fc 100644 --- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkDDLDataTypeTest.java +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkDDLDataTypeTest.java @@ -21,10 +21,11 @@ package org.apache.flink.sql.parser; import org.apache.flink.sql.parser.ddl.SqlCreateTable; import org.apache.flink.sql.parser.ddl.SqlTableColumn; import org.apache.flink.sql.parser.impl.FlinkSqlParserImpl; -import org.apache.flink.sql.parser.validate.FlinkSqlConformance; import org.apache.calcite.avatica.util.Casing; import org.apache.calcite.avatica.util.Quoting; +import org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.calcite.rel.type.DelegatingTypeSystem; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeSystem; @@ -32,16 +33,27 @@ import org.apache.calcite.sql.SqlDataTypeSpec; import org.apache.calcite.sql.SqlDialect; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlOperatorTable; import org.apache.calcite.sql.SqlWriter; import org.apache.calcite.sql.dialect.CalciteSqlDialect; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; +import org.apache.calcite.sql.parser.SqlParserImplFactory; import org.apache.calcite.sql.parser.SqlParserTest; import org.apache.calcite.sql.parser.SqlParserUtil; import org.apache.calcite.sql.pretty.SqlPrettyWriter; -import org.apache.calcite.sql.type.SqlTypeFactoryImpl; +import org.apache.calcite.sql.test.SqlTestFactory; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.sql.validate.SqlConformance; +import org.apache.calcite.sql.validate.SqlConformanceEnum; +import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.sql.validate.SqlValidatorCatalogReader; +import org.apache.calcite.sql.validate.SqlValidatorUtil; +import org.apache.calcite.test.MockSqlOperatorTable; import org.apache.calcite.test.SqlValidatorTestCase; +import org.apache.calcite.test.catalog.MockCatalogReaderSimple; +import org.apache.calcite.util.SourceStringReader; import org.apache.calcite.util.Util; import org.junit.Test; import org.junit.runner.RunWith; @@ -51,19 +63,19 @@ import javax.annotation.Nullable; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static org.junit.Assert.assertEquals; /** - * Tests for all the sup[ported flink DDL data types. + * Tests for all the supported Flink DDL data types. */ @RunWith(Parameterized.class) public class FlinkDDLDataTypeTest { - private FlinkSqlConformance conformance = FlinkSqlConformance.DEFAULT; - private static final RelDataTypeFactory TYPE_FACTORY = - new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT); - private static final Fixture FIXTURE = new Fixture(TYPE_FACTORY); + private static final Fixture FIXTURE = + new Fixture(TestFactory.INSTANCE.getTypeFactory()); private static final String DDL_FORMAT = "create table t1 (\n" + " f0 %s\n" + ") with (\n" + @@ -171,7 +183,7 @@ public class FlinkDDLDataTypeTest { nullable(FIXTURE.createStructType( Arrays.asList(FIXTURE.intType, nullable(FIXTURE.booleanType)), Arrays.asList("f0", "f1"))), - "ROW< `f0` INTEGER NOT NULL, `f1` BOOLEAN >"), + "ROW(`f0` INTEGER NOT NULL, `f1` BOOLEAN)"), createTestItem("ROW<`f0` INT>", nullable(FIXTURE.createStructType( Collections.singletonList(nullable(FIXTURE.intType)), @@ -181,7 +193,7 @@ public class FlinkDDLDataTypeTest { nullable(FIXTURE.createStructType( Collections.singletonList(nullable(FIXTURE.intType)), Collections.singletonList("f0"))), - "ROW< `f0` INTEGER >"), + "ROW(`f0` INTEGER)"), createTestItem("ROW<>", nullable(FIXTURE.createStructType( Collections.emptyList(), @@ -191,7 +203,7 @@ public class FlinkDDLDataTypeTest { nullable(FIXTURE.createStructType( Collections.emptyList(), Collections.emptyList())), - "ROW<>"), + "ROW()"), createTestItem("ROW", nullable(FIXTURE.createStructType( @@ -200,25 +212,19 @@ public class FlinkDDLDataTypeTest { "ROW< `f0` INTEGER NOT NULL 'This is a comment.', " + "`f1` BOOLEAN 'This as well.' >"), - // test parse throws error. - createTestItem("TIMESTAMP WITH TIME ZONE", - "'WITH TIME ZONE' is not supported yet, options: " - + "'WITHOUT TIME ZONE', 'WITH LOCAL TIME ZONE'."), - createTestItem("TIMESTAMP(3) WITH TIME ZONE", - "'WITH TIME ZONE' is not supported yet, options: " - + "'WITHOUT TIME ZONE', 'WITH LOCAL TIME ZONE'."), + // Test parse throws error. + createTestItem("TIMESTAMP WITH ^TIME^ ZONE", + "(?s).*Encountered \"TIME\" at .*"), + createTestItem("TIMESTAMP(3) WITH ^TIME^ ZONE", + "(?s).*Encountered \"TIME\" at .*"), createTestItem("^NULL^", "(?s).*Encountered \"NULL\" at line 2, column 6..*"), - createTestItem("cat.db.MyType", - "(?s).*UDT in DDL is not supported yet..*"), - createTestItem("`db`.`MyType`", - "(?s).*UDT in DDL is not supported yet..*"), - createTestItem("MyType", - "(?s).*UDT in DDL is not supported yet..*"), - createTestItem("ARRAY", - "(?s).*UDT in DDL is not supported yet..*"), - createTestItem("ROW", - "(?s).*UDT in DDL is not supported yet..*"), + createTestItem("cat.db.MyType", null, "`cat`.`db`.`MyType`"), + createTestItem("`db`.`MyType`", null, "`db`.`MyType`"), + createTestItem("MyType", null, "`MyType`"), + createTestItem("ARRAY", null, "ARRAY< `MyType` >"), + createTestItem("ROW", null, + "ROW< `f0` `MyType`, `f1` `c`.`d`.`t` >"), createTestItem("^INTERVAL^ YEAR", "(?s).*Encountered \"INTERVAL\" at line 2, column 6..*"), createTestItem("ANY(^'unknown.class'^, '')", @@ -284,7 +290,7 @@ public class FlinkDDLDataTypeTest { } private Tester getTester() { - return new TesterImpl(); + return new TesterImpl(TestFactory.INSTANCE); } private Sql sql(String sql) { @@ -369,23 +375,10 @@ public class FlinkDDLDataTypeTest { * Default implementation of {@link SqlParserTest.Tester}. */ protected class TesterImpl implements Tester { - private SqlParser getSqlParser(String sql) { - return SqlParser.create(sql, - SqlParser.configBuilder() - .setParserFactory(FlinkSqlParserImpl.FACTORY) - .setQuoting(Quoting.BACK_TICK) - .setUnquotedCasing(Casing.UNCHANGED) - .setQuotedCasing(Casing.UNCHANGED) - .setConformance(conformance) - .build()); - } + private TestFactory factory; - private SqlDialect getSqlDialect() { - return new CalciteSqlDialect(SqlDialect.EMPTY_CONTEXT - .withQuotedCasing(Casing.UNCHANGED) - .withConformance(conformance) - .withUnquotedCasing(Casing.UNCHANGED) - .withIdentifierQuoteString("`")); + TesterImpl(TestFactory factory) { + this.factory = factory; } public void checkType(String sql, RelDataType type) { @@ -395,14 +388,14 @@ public class FlinkDDLDataTypeTest { SqlNodeList columns = sqlCreateTable.getColumnList(); assert columns.size() == 1; RelDataType columnType = ((SqlTableColumn) columns.get(0)).getType() - .deriveType(TYPE_FACTORY); + .deriveType(factory.getValidator()); assertEquals(type, columnType); } private SqlNode parseStmtAndHandleEx(String sql) { final SqlNode sqlNode; try { - sqlNode = getSqlParser(sql).parseStmt(); + sqlNode = factory.createParser(sql).parseStmt(); } catch (SqlParseException e) { throw new RuntimeException("Error while parsing SQL: " + sql, e); } @@ -416,7 +409,7 @@ public class FlinkDDLDataTypeTest { Throwable thrown = null; try { final SqlNode sqlNode; - sqlNode = getSqlParser(sap.sql).parseStmt(); + sqlNode = factory.createParser(sap.sql).parseStmt(); Util.discard(sqlNode); } catch (Throwable ex) { thrown = ex; @@ -432,8 +425,14 @@ public class FlinkDDLDataTypeTest { SqlNodeList columns = sqlCreateTable.getColumnList(); assert columns.size() == 1; SqlDataTypeSpec dataTypeSpec = ((SqlTableColumn) columns.get(0)).getType(); - SqlWriter sqlWriter = new SqlPrettyWriter(getSqlDialect(), false); + SqlWriter sqlWriter = new SqlPrettyWriter(factory.createSqlDialect(), false); dataTypeSpec.unparse(sqlWriter, 0, 0); + // SqlDataTypeSpec does not take care of the nullable attribute unparse, + // So we unparse nullable attribute specifically, this unparsing logic should + // keep sync with SqlTableColumn. + if (!dataTypeSpec.getNullable()) { + sqlWriter.keyword("NOT NULL"); + } assertEquals(expectedUnparsed, sqlWriter.toSqlString().getSql()); } @@ -443,4 +442,115 @@ public class FlinkDDLDataTypeTest { SqlValidatorTestCase.checkEx(thrown, expectedMsgPattern, sap); } } + + /** + * Factory to supply test components. + */ + private static class TestFactory { + static final Map DEFAULT_OPTIONS = buildDefaultOptions(); + + public static final TestFactory INSTANCE = new TestFactory(); + + private final Map options; + private final SqlTestFactory.ValidatorFactory validatorFactory; + + private final RelDataTypeFactory typeFactory; + private final SqlOperatorTable operatorTable; + private final SqlValidatorCatalogReader catalogReader; + private final SqlParser.Config parserConfig; + + TestFactory() { + this(DEFAULT_OPTIONS, MockCatalogReaderSimple::new, SqlValidatorUtil::newValidator); + } + + TestFactory( + Map options, + SqlTestFactory.MockCatalogReaderFactory catalogReaderFactory, + SqlTestFactory.ValidatorFactory validatorFactory) { + this.options = options; + this.validatorFactory = validatorFactory; + this.operatorTable = + createOperatorTable((SqlOperatorTable) options.get("operatorTable")); + this.typeFactory = createTypeFactory((SqlConformance) options.get("conformance")); + Boolean caseSensitive = (Boolean) options.get("caseSensitive"); + this.catalogReader = catalogReaderFactory.create(typeFactory, caseSensitive).init(); + this.parserConfig = createParserConfig(options); + } + + public SqlParser createParser(String sql) { + return SqlParser.create(new SourceStringReader(sql), parserConfig); + } + + public SqlDialect createSqlDialect() { + return new CalciteSqlDialect(SqlDialect.EMPTY_CONTEXT + .withQuotedCasing(parserConfig.unquotedCasing()) + .withConformance(parserConfig.conformance()) + .withUnquotedCasing(parserConfig.unquotedCasing()) + .withIdentifierQuoteString(parserConfig.quoting().string)); + } + + public RelDataTypeFactory getTypeFactory() { + return this.typeFactory; + } + + public SqlValidator getValidator() { + final SqlConformance conformance = + (SqlConformance) options.get("conformance"); + final boolean enableTypeCoercion = (boolean) options.get("enableTypeCoercion"); + return validatorFactory.create(operatorTable, + catalogReader, + typeFactory, + conformance) + .setEnableTypeCoercion(enableTypeCoercion); + } + + private static SqlOperatorTable createOperatorTable(SqlOperatorTable opTab0) { + MockSqlOperatorTable opTab = new MockSqlOperatorTable(opTab0); + MockSqlOperatorTable.addRamp(opTab); + return opTab; + } + + private static SqlParser.Config createParserConfig(Map options) { + return SqlParser.configBuilder() + .setQuoting((Quoting) options.get("quoting")) + .setUnquotedCasing((Casing) options.get("unquotedCasing")) + .setQuotedCasing((Casing) options.get("quotedCasing")) + .setConformance((SqlConformance) options.get("conformance")) + .setCaseSensitive((boolean) options.get("caseSensitive")) + .setParserFactory((SqlParserImplFactory) options.get("parserFactory")) + .build(); + } + + private static RelDataTypeFactory createTypeFactory(SqlConformance conformance) { + RelDataTypeSystem typeSystem = RelDataTypeSystem.DEFAULT; + if (conformance.shouldConvertRaggedUnionTypesToVarying()) { + typeSystem = new DelegatingTypeSystem(typeSystem) { + public boolean shouldConvertRaggedUnionTypesToVarying() { + return true; + } + }; + } + if (conformance.allowExtendedTrim()) { + typeSystem = new DelegatingTypeSystem(typeSystem) { + public boolean allowExtendedTrim() { + return true; + } + }; + } + return new JavaTypeFactoryImpl(typeSystem); + } + + private static Map buildDefaultOptions() { + final Map m = new HashMap<>(); + m.put("quoting", Quoting.BACK_TICK); + m.put("quotedCasing", Casing.UNCHANGED); + m.put("unquotedCasing", Casing.UNCHANGED); + m.put("caseSensitive", true); + m.put("enableTypeCoercion", false); + m.put("conformance", SqlConformanceEnum.DEFAULT); + m.put("operatorTable", SqlStdOperatorTable.instance()); + m.put("parserFactory", FlinkSqlParserImpl.FACTORY); + return Collections.unmodifiableMap(m); + } + } } diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java index 7a494b3e84..36c742a2ad 100644 --- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java @@ -416,8 +416,14 @@ public class FlinkSqlParserImplTest extends SqlParserTest { " 'k1' = 'v1',\n" + " 'k2' = 'v2'\n" + ")"; - final String errMsg = "UDT in DDL is not supported yet."; - checkFails(sql, errMsg); + final String expected = "CREATE TABLE `T` (\n" + + " `A` `CATALOG1`.`DB1`.`MYTYPE1`,\n" + + " `B` `DB2`.`MYTYPE2`\n" + + ") WITH (\n" + + " 'k1' = 'v1',\n" + + " 'k2' = 'v2'\n" + + ")"; + check(sql, expected); } @Test @@ -433,7 +439,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { ")\n", "(?s).*Encountered \"\\(\" at line 4, column 14.\n" + "Was expecting one of:\n" + " \"AS\" ...\n" + - " \"ARRAY\" ...\n" + + " \"STRING\" ...\n" + ".*"); } @@ -663,7 +669,25 @@ public class FlinkSqlParserImplTest extends SqlParserTest { check(sql, "DROP VIEW IF EXISTS `VIEW_NAME`"); } - /** Matcher that invokes the #validate() of the produced SqlNode. **/ + // Override the test because our ROW field type default is nullable, + // which is different with Calcite. + @Override + public void testCastAsRowType() { + checkExp("cast(a as row(f0 int, f1 varchar))", + "CAST(`A` AS ROW(`F0` INTEGER, `F1` VARCHAR))"); + checkExp("cast(a as row(f0 int not null, f1 varchar null))", + "CAST(`A` AS ROW(`F0` INTEGER NOT NULL, `F1` VARCHAR))"); + checkExp("cast(a as row(f0 row(ff0 int not null, ff1 varchar null) null," + + " f1 timestamp not null))", + "CAST(`A` AS ROW(`F0` ROW(`FF0` INTEGER NOT NULL, `FF1` VARCHAR)," + + " `F1` TIMESTAMP NOT NULL))"); + checkExp("cast(a as row(f0 bigint not null, f1 decimal null) array)", + "CAST(`A` AS ROW(`F0` BIGINT NOT NULL, `F1` DECIMAL) ARRAY)"); + checkExp("cast(a as row(f0 varchar not null, f1 timestamp null) multiset)", + "CAST(`A` AS ROW(`F0` VARCHAR NOT NULL, `F1` TIMESTAMP) MULTISET)"); + } + + /** Matcher that invokes the #validate() of the {@link ExtendedSqlNode} instance. **/ private static class ValidationMatcher extends BaseMatcher { private String expectedColumnSql; private String failMsg; diff --git a/flink-table/pom.xml b/flink-table/pom.xml index 1e479b3523..400cd17a3e 100644 --- a/flink-table/pom.xml +++ b/flink-table/pom.xml @@ -50,5 +50,6 @@ under the License. 3.0.9 + 1.21.0 -- Gitee From 4f0d8044ca7dd0774dcb9b66a9faf9fbe152d187 Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Fri, 27 Sep 2019 09:40:39 +0800 Subject: [PATCH 074/268] [FLINK-13656][table-planner][table-planner-blink] Bump flink and blink planner pom and notice file Calcite version to 1.21.0 --- NOTICE-binary | 8 ++++---- flink-table/flink-table-planner-blink/pom.xml | 8 ++++---- .../src/main/resources/META-INF/NOTICE | 4 ++-- flink-table/flink-table-planner/pom.xml | 8 ++++---- .../src/main/resources/META-INF/NOTICE | 4 ++-- 5 files changed, 16 insertions(+), 16 deletions(-) diff --git a/NOTICE-binary b/NOTICE-binary index f794ace44b..122242c180 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -16537,8 +16537,8 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-databind:2.9.6 - com.jayway.jsonpath:json-path:2.4.0 - joda-time:joda-time:2.5 -- org.apache.calcite:calcite-core:1.20.0 -- org.apache.calcite:calcite-linq4j:1.20.0 +- org.apache.calcite:calcite-core:1.21.0 +- org.apache.calcite:calcite-linq4j:1.21.0 - org.apache.calcite.avatica:avatica-core:1.15.0 - commons-codec:commons-codec:1.10 @@ -16651,8 +16651,8 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-databind:2.9.6 - com.jayway.jsonpath:json-path:2.4.0 - joda-time:joda-time:2.5 -- org.apache.calcite:calcite-core:1.20.0 -- org.apache.calcite:calcite-linq4j:1.20.0 +- org.apache.calcite:calcite-core:1.21.0 +- org.apache.calcite:calcite-linq4j:1.21.0 - org.apache.calcite.avatica:avatica-core:1.15.0 - commons-codec:commons-codec:1.10 diff --git a/flink-table/flink-table-planner-blink/pom.xml b/flink-table/flink-table-planner-blink/pom.xml index f9e7f2b73b..1d60b7ad18 100644 --- a/flink-table/flink-table-planner-blink/pom.xml +++ b/flink-table/flink-table-planner-blink/pom.xml @@ -162,14 +162,14 @@ under the License. org.apache.calcite calcite-core - 1.20.0 + ${calcite.version} - 1.20.0 + ${calcite.version} newInput - // - // newProject - // | - // oldInput ----> newInput - // - // is transformed to - // - // oldInput ----> newProject - // | - // newInput - Map combinedMap = new HashMap<>(); - - for (Integer oldInputPos : frame.oldToNewOutputs.keySet()) { - combinedMap.put(oldInputPos, - mapNewInputToProjOutputs.get( - frame.oldToNewOutputs.get(oldInputPos))); - } - - register(oldInput, newProject, combinedMap, corDefOutputs); - - // now it's time to rewrite the Aggregate - final ImmutableBitSet newGroupSet = ImmutableBitSet.range(newGroupKeyCount); - List newAggCalls = new ArrayList<>(); - List oldAggCalls = rel.getAggCallList(); - - ImmutableList newGroupSets = null; - if (rel.getGroupType() != Aggregate.Group.SIMPLE) { - final ImmutableBitSet addedGroupSet = - ImmutableBitSet.range(oldGroupKeyCount, newGroupKeyCount); - final Iterable tmpGroupSets = - Iterables.transform(rel.getGroupSets(), - bitSet -> bitSet.union(addedGroupSet)); - newGroupSets = ImmutableBitSet.ORDERING.immutableSortedCopy(tmpGroupSets); - } - - int oldInputOutputFieldCount = rel.getGroupSet().cardinality(); - int newInputOutputFieldCount = newGroupSet.cardinality(); - - int i = -1; - for (AggregateCall oldAggCall : oldAggCalls) { - ++i; - List oldAggArgs = oldAggCall.getArgList(); - - List aggArgs = new ArrayList<>(); - - // Adjust the Aggregate argument positions. - // Note Aggregate does not change input ordering, so the input - // output position mapping can be used to derive the new positions - // for the argument. - for (int oldPos : oldAggArgs) { - aggArgs.add(combinedMap.get(oldPos)); - } - final int filterArg = oldAggCall.filterArg < 0 ? oldAggCall.filterArg - : combinedMap.get(oldAggCall.filterArg); - - newAggCalls.add( - oldAggCall.adaptTo(newProject, aggArgs, filterArg, - oldGroupKeyCount, newGroupKeyCount)); - - // The old to new output position mapping will be the same as that - // of newProject, plus any aggregates that the oldAgg produces. - outputMap.put( - oldInputOutputFieldCount + i, - newInputOutputFieldCount + i); - } - - relBuilder.push( - LogicalAggregate.create(newProject, newGroupSet, newGroupSets, newAggCalls)); - - if (!omittedConstants.isEmpty()) { - final List postProjects = new ArrayList<>(relBuilder.fields()); - for (Map.Entry entry - : omittedConstants.descendingMap().entrySet()) { - int index = entry.getKey() + frame.corDefOutputs.size(); - postProjects.add(index, entry.getValue()); - // Shift the outputs whose index equals with or bigger than the added index - // with 1 offset. - shiftMapping(outputMap, index, 1); - // Then add the constant key mapping. - outputMap.put(entry.getKey(), index); - } - relBuilder.project(postProjects); - } - - // Aggregate does not change input ordering so corVars will be - // located at the same position as the input newProject. - return register(rel, relBuilder.build(), outputMap, corDefOutputs); - } - - /** - * Shift the mapping to fixed offset from the {@code startIndex}. - * @param mapping the original mapping - * @param startIndex any output whose index equals with or bigger than the starting index - * would be shift - * @param offset shift offset - */ - private static void shiftMapping(Map mapping, int startIndex, int offset) { - for (Map.Entry entry : mapping.entrySet()) { - if (entry.getValue() >= startIndex) { - mapping.put(entry.getKey(), entry.getValue() + offset); - } else { - mapping.put(entry.getKey(), entry.getValue()); - } - } - } - - public Frame getInvoke(RelNode r, RelNode parent) { - final Frame frame = dispatcher.invoke(r); - if (frame != null) { - map.put(r, frame); - } - currentRel = parent; - return frame; - } - - /** Returns a literal output field, or null if it is not literal. */ - private static RexLiteral projectedLiteral(RelNode rel, int i) { - if (rel instanceof Project) { - final Project project = (Project) rel; - final RexNode node = project.getProjects().get(i); - if (node instanceof RexLiteral) { - return (RexLiteral) node; - } - } - return null; - } - - /** - * Rewrite LogicalProject. - * - * @param rel the project rel to rewrite - */ - public Frame decorrelateRel(LogicalProject rel) { - // - // Rewrite logic: - // - // 1. Pass along any correlated variables coming from the input. - // - - final RelNode oldInput = rel.getInput(); - Frame frame = getInvoke(oldInput, rel); - if (frame == null) { - // If input has not been rewritten, do not rewrite this rel. - return null; - } - final List oldProjects = rel.getProjects(); - final List relOutput = rel.getRowType().getFieldList(); - - // Project projects the original expressions, - // plus any correlated variables the input wants to pass along. - final List> projects = new ArrayList<>(); - - // If this Project has correlated reference, create value generator - // and produce the correlated variables in the new output. - if (cm.mapRefRelToCorRef.containsKey(rel)) { - frame = decorrelateInputWithValueGenerator(rel, frame); - } - - // Project projects the original expressions - final Map mapOldToNewOutputs = new HashMap<>(); - int newPos; - for (newPos = 0; newPos < oldProjects.size(); newPos++) { - projects.add( - newPos, - Pair.of( - decorrelateExpr(currentRel, map, cm, oldProjects.get(newPos)), - relOutput.get(newPos).getName())); - mapOldToNewOutputs.put(newPos, newPos); - } - - // Project any correlated variables the input wants to pass along. - final SortedMap corDefOutputs = new TreeMap<>(); - for (Map.Entry entry : frame.corDefOutputs.entrySet()) { - projects.add( - RexInputRef.of2(entry.getValue(), - frame.r.getRowType().getFieldList())); - corDefOutputs.put(entry.getKey(), newPos); - newPos++; - } - - RelNode newProject = relBuilder.push(frame.r) - .projectNamed(Pair.left(projects), Pair.right(projects), true) - .build(); - - return register(rel, newProject, mapOldToNewOutputs, corDefOutputs); - } - - /** - * Create RelNode tree that produces a list of correlated variables. - * - * @param correlations correlated variables to generate - * @param valueGenFieldOffset offset in the output that generated columns - * will start - * @param corDefOutputs output positions for the correlated variables - * generated - * @return RelNode the root of the resultant RelNode tree - */ - private RelNode createValueGenerator( - Iterable correlations, - int valueGenFieldOffset, - SortedMap corDefOutputs) { - final Map> mapNewInputToOutputs = new HashMap<>(); - - final Map mapNewInputToNewOffset = new HashMap<>(); - - // Input provides the definition of a correlated variable. - // Add to map all the referenced positions (relative to each input rel). - for (CorRef corVar : correlations) { - final int oldCorVarOffset = corVar.field; - - final RelNode oldInput = getCorRel(corVar); - assert oldInput != null; - final Frame frame = getFrame(oldInput, true); - assert frame != null; - final RelNode newInput = frame.r; - - final List newLocalOutputs; - if (!mapNewInputToOutputs.containsKey(newInput)) { - newLocalOutputs = new ArrayList<>(); - } else { - newLocalOutputs = mapNewInputToOutputs.get(newInput); - } - - final int newCorVarOffset = frame.oldToNewOutputs.get(oldCorVarOffset); - - // Add all unique positions referenced. - if (!newLocalOutputs.contains(newCorVarOffset)) { - newLocalOutputs.add(newCorVarOffset); - } - mapNewInputToOutputs.put(newInput, newLocalOutputs); - } - - int offset = 0; - - // Project only the correlated fields out of each input - // and join the project together. - // To make sure the plan does not change in terms of join order, - // join these rels based on their occurrence in corVar list which - // is sorted. - final Set joinedInputs = new HashSet<>(); - - RelNode r = null; - for (CorRef corVar : correlations) { - final RelNode oldInput = getCorRel(corVar); - assert oldInput != null; - final RelNode newInput = getFrame(oldInput, true).r; - assert newInput != null; - - if (!joinedInputs.contains(newInput)) { - RelNode project = - RelOptUtil.createProject(newInput, - mapNewInputToOutputs.get(newInput)); - RelNode distinct = relBuilder.push(project) - .distinct() - .build(); - RelOptCluster cluster = distinct.getCluster(); - - joinedInputs.add(newInput); - mapNewInputToNewOffset.put(newInput, offset); - offset += distinct.getRowType().getFieldCount(); - - if (r == null) { - r = distinct; - } else { - r = - LogicalJoin.create(r, distinct, - cluster.getRexBuilder().makeLiteral(true), - ImmutableSet.of(), JoinRelType.INNER); - } - } - } - - // Translate the positions of correlated variables to be relative to - // the join output, leaving room for valueGenFieldOffset because - // valueGenerators are joined with the original left input of the rel - // referencing correlated variables. - for (CorRef corRef : correlations) { - // The first input of a Correlate is always the rel defining - // the correlated variables. - final RelNode oldInput = getCorRel(corRef); - assert oldInput != null; - final Frame frame = getFrame(oldInput, true); - final RelNode newInput = frame.r; - assert newInput != null; - - final List newLocalOutputs = mapNewInputToOutputs.get(newInput); - - final int newLocalOutput = frame.oldToNewOutputs.get(corRef.field); - - // newOutput is the index of the corVar in the referenced - // position list plus the offset of referenced position list of - // each newInput. - final int newOutput = - newLocalOutputs.indexOf(newLocalOutput) - + mapNewInputToNewOffset.get(newInput) - + valueGenFieldOffset; - - corDefOutputs.put(corRef.def(), newOutput); - } - - return r; - } - - private Frame getFrame(RelNode r, boolean safe) { - final Frame frame = map.get(r); - if (frame == null && safe) { - return new Frame(r, r, ImmutableSortedMap.of(), - identityMap(r.getRowType().getFieldCount())); - } - return frame; - } - - private RelNode getCorRel(CorRef corVar) { - final RelNode r = cm.mapCorToCorRel.get(corVar.corr); - return r.getInput(0); - } - - /** Adds a value generator to satisfy the correlating variables used by - * a relational expression, if those variables are not already provided by - * its input. */ - private Frame maybeAddValueGenerator(RelNode rel, Frame frame) { - final CorelMap cm1 = new CorelMapBuilder().build(frame.r, rel); - if (!cm1.mapRefRelToCorRef.containsKey(rel)) { - return frame; - } - final Collection needs = cm1.mapRefRelToCorRef.get(rel); - final ImmutableSortedSet haves = frame.corDefOutputs.keySet(); - if (hasAll(needs, haves)) { - return frame; - } - return decorrelateInputWithValueGenerator(rel, frame); - } - - /** Returns whether all of a collection of {@link CorRef}s are satisfied - * by at least one of a collection of {@link CorDef}s. */ - private boolean hasAll(Collection corRefs, - Collection corDefs) { - for (CorRef corRef : corRefs) { - if (!has(corDefs, corRef)) { - return false; - } - } - return true; - } - - /** Returns whether a {@link CorrelationId} is satisfied by at least one of a - * collection of {@link CorDef}s. */ - private boolean has(Collection corDefs, CorRef corr) { - for (CorDef corDef : corDefs) { - if (corDef.corr.equals(corr.corr) && corDef.field == corr.field) { - return true; - } - } - return false; - } - - private Frame decorrelateInputWithValueGenerator(RelNode rel, Frame frame) { - // currently only handles one input - assert rel.getInputs().size() == 1; - RelNode oldInput = frame.r; - - final SortedMap corDefOutputs = - new TreeMap<>(frame.corDefOutputs); - - final Collection corVarList = cm.mapRefRelToCorRef.get(rel); - - // Try to populate correlation variables using local fields. - // This means that we do not need a value generator. - if (rel instanceof Filter) { - SortedMap map = new TreeMap<>(); - List projects = new ArrayList<>(); - for (CorRef correlation : corVarList) { - final CorDef def = correlation.def(); - if (corDefOutputs.containsKey(def) || map.containsKey(def)) { - continue; - } - try { - findCorrelationEquivalent(correlation, ((Filter) rel).getCondition()); - } catch (Util.FoundOne e) { - if (e.getNode() instanceof RexInputRef) { - map.put(def, ((RexInputRef) e.getNode()).getIndex()); - } else { - map.put(def, - frame.r.getRowType().getFieldCount() + projects.size()); - projects.add((RexNode) e.getNode()); - } - } - } - // If all correlation variables are now satisfied, skip creating a value - // generator. - if (map.size() == corVarList.size()) { - map.putAll(frame.corDefOutputs); - final RelNode r; - if (!projects.isEmpty()) { - relBuilder.push(oldInput) - .project(Iterables.concat(relBuilder.fields(), projects)); - r = relBuilder.build(); - } else { - r = oldInput; - } - return register(rel.getInput(0), r, - frame.oldToNewOutputs, map); - } - } - - int leftInputOutputCount = frame.r.getRowType().getFieldCount(); - - // can directly add positions into corDefOutputs since join - // does not change the output ordering from the inputs. - RelNode valueGen = - createValueGenerator(corVarList, leftInputOutputCount, corDefOutputs); - - RelNode join = - LogicalJoin.create(frame.r, valueGen, relBuilder.literal(true), - ImmutableSet.of(), JoinRelType.INNER); - - // Join or Filter does not change the old input ordering. All - // input fields from newLeftInput (i.e. the original input to the old - // Filter) are in the output and in the same position. - return register(rel.getInput(0), join, frame.oldToNewOutputs, - corDefOutputs); - } - - /** Finds a {@link RexInputRef} that is equivalent to a {@link CorRef}, - * and if found, throws a {@link org.apache.calcite.util.Util.FoundOne}. */ - private void findCorrelationEquivalent(CorRef correlation, RexNode e) - throws Util.FoundOne { - switch (e.getKind()) { - case EQUALS: - final RexCall call = (RexCall) e; - final List operands = call.getOperands(); - if (references(operands.get(0), correlation)) { - throw new Util.FoundOne(operands.get(1)); - } - if (references(operands.get(1), correlation)) { - throw new Util.FoundOne(operands.get(0)); - } - break; - case AND: - for (RexNode operand : ((RexCall) e).getOperands()) { - findCorrelationEquivalent(correlation, operand); - } - } - } - - private boolean references(RexNode e, CorRef correlation) { - switch (e.getKind()) { - case CAST: - final RexNode operand = ((RexCall) e).getOperands().get(0); - if (isWidening(e.getType(), operand.getType())) { - return references(operand, correlation); - } - return false; - case FIELD_ACCESS: - final RexFieldAccess f = (RexFieldAccess) e; - if (f.getField().getIndex() == correlation.field - && f.getReferenceExpr() instanceof RexCorrelVariable) { - if (((RexCorrelVariable) f.getReferenceExpr()).id == correlation.corr) { - return true; - } - } - // fall through - default: - return false; - } - } - - /** Returns whether one type is just a widening of another. - * - *

For example:

    - *
  • {@code VARCHAR(10)} is a widening of {@code VARCHAR(5)}. - *
  • {@code VARCHAR(10)} is a widening of {@code VARCHAR(10) NOT NULL}. - *
- */ - private boolean isWidening(RelDataType type, RelDataType type1) { - return type.getSqlTypeName() == type1.getSqlTypeName() - && type.getPrecision() >= type1.getPrecision(); - } - - /** - * Rewrite LogicalSnapshot. - * - * @param rel the snapshot rel to rewrite - */ - public Frame decorrelateRel(LogicalSnapshot rel) { - if (RexUtil.containsCorrelation(rel.getPeriod())) { - return null; - } - return decorrelateRel((RelNode) rel); - } - - /** - * Rewrite LogicalFilter. - * - * @param rel the filter rel to rewrite - */ - public Frame decorrelateRel(LogicalFilter rel) { - // - // Rewrite logic: - // - // 1. If a Filter references a correlated field in its filter - // condition, rewrite the Filter to be - // Filter - // Join(cross product) - // originalFilterInput - // ValueGenerator(produces distinct sets of correlated variables) - // and rewrite the correlated fieldAccess in the filter condition to - // reference the Join output. - // - // 2. If Filter does not reference correlated variables, simply - // rewrite the filter condition using new input. - // - - final RelNode oldInput = rel.getInput(); - Frame frame = getInvoke(oldInput, rel); - if (frame == null) { - // If input has not been rewritten, do not rewrite this rel. - return null; - } - - // If this Filter has correlated reference, create value generator - // and produce the correlated variables in the new output. - if (false) { - if (cm.mapRefRelToCorRef.containsKey(rel)) { - frame = decorrelateInputWithValueGenerator(rel, frame); - } - } else { - frame = maybeAddValueGenerator(rel, frame); - } - - final CorelMap cm2 = new CorelMapBuilder().build(rel); - - // Replace the filter expression to reference output of the join - // Map filter to the new filter over join - relBuilder.push(frame.r) - .filter(decorrelateExpr(currentRel, map, cm2, rel.getCondition())); - - // Filter does not change the input ordering. - // Filter rel does not permute the input. - // All corVars produced by filter will have the same output positions in the - // input rel. - return register(rel, relBuilder.build(), frame.oldToNewOutputs, - frame.corDefOutputs); - } - - /** - * Rewrite Correlate into a left outer join. - * - * @param rel Correlator - */ - public Frame decorrelateRel(LogicalCorrelate rel) { - // - // Rewrite logic: - // - // The original left input will be joined with the new right input that - // has generated correlated variables propagated up. For any generated - // corVars that are not used in the join key, pass them along to be - // joined later with the Correlates that produce them. - // - - // the right input to Correlate should produce correlated variables - final RelNode oldLeft = rel.getInput(0); - final RelNode oldRight = rel.getInput(1); - - final Frame leftFrame = getInvoke(oldLeft, rel); - final Frame rightFrame = getInvoke(oldRight, rel); - - if (leftFrame == null || rightFrame == null) { - // If any input has not been rewritten, do not rewrite this rel. - return null; - } - - if (rightFrame.corDefOutputs.isEmpty()) { - return null; - } - - assert rel.getRequiredColumns().cardinality() - <= rightFrame.corDefOutputs.keySet().size(); - - // Change correlator rel into a join. - // Join all the correlated variables produced by this correlator rel - // with the values generated and propagated from the right input - final SortedMap corDefOutputs = - new TreeMap<>(rightFrame.corDefOutputs); - final List conditions = new ArrayList<>(); - final List newLeftOutput = - leftFrame.r.getRowType().getFieldList(); - int newLeftFieldCount = newLeftOutput.size(); - - final List newRightOutput = - rightFrame.r.getRowType().getFieldList(); - - for (Map.Entry rightOutput - : new ArrayList<>(corDefOutputs.entrySet())) { - final CorDef corDef = rightOutput.getKey(); - if (!corDef.corr.equals(rel.getCorrelationId())) { - continue; - } - final int newLeftPos = leftFrame.oldToNewOutputs.get(corDef.field); - final int newRightPos = rightOutput.getValue(); - conditions.add( - relBuilder.call(SqlStdOperatorTable.EQUALS, - RexInputRef.of(newLeftPos, newLeftOutput), - new RexInputRef(newLeftFieldCount + newRightPos, - newRightOutput.get(newRightPos).getType()))); - - // remove this corVar from output position mapping - corDefOutputs.remove(corDef); - } - - // Update the output position for the corVars: only pass on the cor - // vars that are not used in the join key. - for (CorDef corDef : corDefOutputs.keySet()) { - int newPos = corDefOutputs.get(corDef) + newLeftFieldCount; - corDefOutputs.put(corDef, newPos); - } - - // then add any corVar from the left input. Do not need to change - // output positions. - corDefOutputs.putAll(leftFrame.corDefOutputs); - - // Create the mapping between the output of the old correlation rel - // and the new join rel - final Map mapOldToNewOutputs = new HashMap<>(); - - int oldLeftFieldCount = oldLeft.getRowType().getFieldCount(); - - int oldRightFieldCount = oldRight.getRowType().getFieldCount(); - //noinspection AssertWithSideEffects - assert rel.getRowType().getFieldCount() - == oldLeftFieldCount + oldRightFieldCount; - - // Left input positions are not changed. - mapOldToNewOutputs.putAll(leftFrame.oldToNewOutputs); - - // Right input positions are shifted by newLeftFieldCount. - for (int i = 0; i < oldRightFieldCount; i++) { - mapOldToNewOutputs.put(i + oldLeftFieldCount, - rightFrame.oldToNewOutputs.get(i) + newLeftFieldCount); - } - - final RexNode condition = - RexUtil.composeConjunction(relBuilder.getRexBuilder(), conditions); - RelNode newJoin = - LogicalJoin.create(leftFrame.r, rightFrame.r, condition, - ImmutableSet.of(), rel.getJoinType()); - - return register(rel, newJoin, mapOldToNewOutputs, corDefOutputs); - } - - /** - * Rewrite LogicalJoin. - * - * @param rel Join - */ - public Frame decorrelateRel(LogicalJoin rel) { - // For SEMI/ANTI join decorrelate it's input directly, - // because the correlate variables can only be propagated from - // the left side, which is not supported yet. - if (!rel.getJoinType().projectsRight()) { - // fix CALCITE-3169 - return decorrelateRel((RelNode) rel); - } - - // - // Rewrite logic: - // - // 1. rewrite join condition. - // 2. map output positions and produce corVars if any. - // - - final RelNode oldLeft = rel.getInput(0); - final RelNode oldRight = rel.getInput(1); - - final Frame leftFrame = getInvoke(oldLeft, rel); - final Frame rightFrame = getInvoke(oldRight, rel); - - if (leftFrame == null || rightFrame == null) { - // If any input has not been rewritten, do not rewrite this rel. - return null; - } - - final RelNode newJoin = - LogicalJoin.create(leftFrame.r, rightFrame.r, - decorrelateExpr(currentRel, map, cm, rel.getCondition()), - ImmutableSet.of(), rel.getJoinType()); - - // Create the mapping between the output of the old correlation rel - // and the new join rel - Map mapOldToNewOutputs = new HashMap<>(); - - int oldLeftFieldCount = oldLeft.getRowType().getFieldCount(); - int newLeftFieldCount = leftFrame.r.getRowType().getFieldCount(); - - int oldRightFieldCount = oldRight.getRowType().getFieldCount(); - //noinspection AssertWithSideEffects - assert rel.getRowType().getFieldCount() - == oldLeftFieldCount + oldRightFieldCount; - - // Left input positions are not changed. - mapOldToNewOutputs.putAll(leftFrame.oldToNewOutputs); - - // Right input positions are shifted by newLeftFieldCount. - for (int i = 0; i < oldRightFieldCount; i++) { - mapOldToNewOutputs.put(i + oldLeftFieldCount, - rightFrame.oldToNewOutputs.get(i) + newLeftFieldCount); - } - - final SortedMap corDefOutputs = - new TreeMap<>(leftFrame.corDefOutputs); - - // Right input positions are shifted by newLeftFieldCount. - for (Map.Entry entry - : rightFrame.corDefOutputs.entrySet()) { - corDefOutputs.put(entry.getKey(), - entry.getValue() + newLeftFieldCount); - } - return register(rel, newJoin, mapOldToNewOutputs, corDefOutputs); - } - - private static RexInputRef getNewForOldInputRef(RelNode currentRel, - Map map, RexInputRef oldInputRef) { - assert currentRel != null; - - int oldOrdinal = oldInputRef.getIndex(); - int newOrdinal = 0; - - // determine which input rel oldOrdinal references, and adjust - // oldOrdinal to be relative to that input rel - RelNode oldInput = null; - - for (RelNode oldInput0 : currentRel.getInputs()) { - RelDataType oldInputType = oldInput0.getRowType(); - int n = oldInputType.getFieldCount(); - if (oldOrdinal < n) { - oldInput = oldInput0; - break; - } - RelNode newInput = map.get(oldInput0).r; - newOrdinal += newInput.getRowType().getFieldCount(); - oldOrdinal -= n; - } - - assert oldInput != null; - - final Frame frame = map.get(oldInput); - assert frame != null; - - // now oldOrdinal is relative to oldInput - int oldLocalOrdinal = oldOrdinal; - - // figure out the newLocalOrdinal, relative to the newInput. - int newLocalOrdinal = oldLocalOrdinal; - - if (!frame.oldToNewOutputs.isEmpty()) { - newLocalOrdinal = frame.oldToNewOutputs.get(oldLocalOrdinal); - } - - newOrdinal += newLocalOrdinal; - - return new RexInputRef(newOrdinal, - frame.r.getRowType().getFieldList().get(newLocalOrdinal).getType()); - } - - /** - * Pulls project above the join from its RHS input. Enforces nullability - * for join output. - * - * @param join Join - * @param project Original project as the right-hand input of the join - * @param nullIndicatorPos Position of null indicator - * @return the subtree with the new Project at the root - */ - private RelNode projectJoinOutputWithNullability( - LogicalJoin join, - LogicalProject project, - int nullIndicatorPos) { - final RelDataTypeFactory typeFactory = join.getCluster().getTypeFactory(); - final RelNode left = join.getLeft(); - final JoinRelType joinType = join.getJoinType(); - - RexInputRef nullIndicator = - new RexInputRef( - nullIndicatorPos, - typeFactory.createTypeWithNullability( - join.getRowType().getFieldList().get(nullIndicatorPos) - .getType(), - true)); - - // now create the new project - List> newProjExprs = new ArrayList<>(); - - // project everything from the LHS and then those from the original - // projRel - List leftInputFields = - left.getRowType().getFieldList(); - - for (int i = 0; i < leftInputFields.size(); i++) { - newProjExprs.add(RexInputRef.of2(i, leftInputFields)); - } - - // Marked where the projected expr is coming from so that the types will - // become nullable for the original projections which are now coming out - // of the nullable side of the OJ. - boolean projectPulledAboveLeftCorrelator = - joinType.generatesNullsOnRight(); - - for (Pair pair : project.getNamedProjects()) { - RexNode newProjExpr = - removeCorrelationExpr( - pair.left, - projectPulledAboveLeftCorrelator, - nullIndicator); - - newProjExprs.add(Pair.of(newProjExpr, pair.right)); - } - - return relBuilder.push(join) - .projectNamed(Pair.left(newProjExprs), Pair.right(newProjExprs), true) - .build(); - } - - /** - * Pulls a {@link Project} above a {@link Correlate} from its RHS input. - * Enforces nullability for join output. - * - * @param correlate Correlate - * @param project the original project as the RHS input of the join - * @param isCount Positions which are calls to the COUNT - * aggregation function - * @return the subtree with the new Project at the root - */ - private RelNode aggregateCorrelatorOutput( - Correlate correlate, - LogicalProject project, - Set isCount) { - final RelNode left = correlate.getLeft(); - final JoinRelType joinType = correlate.getJoinType(); - - // now create the new project - final List> newProjects = new ArrayList<>(); - - // Project everything from the LHS and then those from the original - // project - final List leftInputFields = - left.getRowType().getFieldList(); - - for (int i = 0; i < leftInputFields.size(); i++) { - newProjects.add(RexInputRef.of2(i, leftInputFields)); - } - - // Marked where the projected expr is coming from so that the types will - // become nullable for the original projections which are now coming out - // of the nullable side of the OJ. - boolean projectPulledAboveLeftCorrelator = - joinType.generatesNullsOnRight(); - - for (Pair pair : project.getNamedProjects()) { - RexNode newProjExpr = - removeCorrelationExpr( - pair.left, - projectPulledAboveLeftCorrelator, - isCount); - newProjects.add(Pair.of(newProjExpr, pair.right)); - } - - return relBuilder.push(correlate) - .projectNamed(Pair.left(newProjects), Pair.right(newProjects), true) - .build(); - } - - /** - * Checks whether the correlations in projRel and filter are related to - * the correlated variables provided by corRel. - * - * @param correlate Correlate - * @param project The original Project as the RHS input of the join - * @param filter Filter - * @param correlatedJoinKeys Correlated join keys - * @return true if filter and proj only references corVar provided by corRel - */ - private boolean checkCorVars( - LogicalCorrelate correlate, - LogicalProject project, - LogicalFilter filter, - List correlatedJoinKeys) { - if (filter != null) { - assert correlatedJoinKeys != null; - - // check that all correlated refs in the filter condition are - // used in the join(as field access). - Set corVarInFilter = - Sets.newHashSet(cm.mapRefRelToCorRef.get(filter)); - - for (RexFieldAccess correlatedJoinKey : correlatedJoinKeys) { - corVarInFilter.remove(cm.mapFieldAccessToCorRef.get(correlatedJoinKey)); - } - - if (!corVarInFilter.isEmpty()) { - return false; - } - - // Check that the correlated variables referenced in these - // comparisons do come from the Correlate. - corVarInFilter.addAll(cm.mapRefRelToCorRef.get(filter)); - - for (CorRef corVar : corVarInFilter) { - if (cm.mapCorToCorRel.get(corVar.corr) != correlate) { - return false; - } - } - } - - // if project has any correlated reference, make sure they are also - // provided by the current correlate. They will be projected out of the LHS - // of the correlate. - if ((project != null) && cm.mapRefRelToCorRef.containsKey(project)) { - for (CorRef corVar : cm.mapRefRelToCorRef.get(project)) { - if (cm.mapCorToCorRel.get(corVar.corr) != correlate) { - return false; - } - } - } - - return true; - } - - /** - * Remove correlated variables from the tree at root corRel - * - * @param correlate Correlate - */ - private void removeCorVarFromTree(LogicalCorrelate correlate) { - if (cm.mapCorToCorRel.get(correlate.getCorrelationId()) == correlate) { - cm.mapCorToCorRel.remove(correlate.getCorrelationId()); - } - } - - /** - * Projects all {@code input} output fields plus the additional expressions. - * - * @param input Input relational expression - * @param additionalExprs Additional expressions and names - * @return the new Project - */ - private RelNode createProjectWithAdditionalExprs( - RelNode input, - List> additionalExprs) { - final List fieldList = - input.getRowType().getFieldList(); - List> projects = new ArrayList<>(); - for (Ord field : Ord.zip(fieldList)) { - projects.add( - Pair.of( - (RexNode) relBuilder.getRexBuilder().makeInputRef( - field.e.getType(), field.i), - field.e.getName())); - } - projects.addAll(additionalExprs); - return relBuilder.push(input) - .projectNamed(Pair.left(projects), Pair.right(projects), true) - .build(); - } - - /* Returns an immutable map with the identity [0: 0, .., count-1: count-1]. */ - static Map identityMap(int count) { - ImmutableMap.Builder builder = ImmutableMap.builder(); - for (int i = 0; i < count; i++) { - builder.put(i, i); - } - return builder.build(); - } - - /** Registers a relational expression and the relational expression it became - * after decorrelation. */ - Frame register(RelNode rel, RelNode newRel, - Map oldToNewOutputs, - SortedMap corDefOutputs) { - final Frame frame = new Frame(rel, newRel, corDefOutputs, oldToNewOutputs); - map.put(rel, frame); - return frame; - } - - static boolean allLessThan(Collection integers, int limit, - Litmus ret) { - for (int value : integers) { - if (value >= limit) { - return ret.fail("out of range; value: {}, limit: {}", value, limit); - } - } - return ret.succeed(); - } - - private static RelNode stripHep(RelNode rel) { - if (rel instanceof HepRelVertex) { - HepRelVertex hepRelVertex = (HepRelVertex) rel; - rel = hepRelVertex.getCurrentRel(); - } - return rel; - } - - //~ Inner Classes ---------------------------------------------------------- - - /** Shuttle that decorrelates. */ - private static class DecorrelateRexShuttle extends RexShuttle { - private final RelNode currentRel; - private final Map map; - private final CorelMap cm; - - private DecorrelateRexShuttle(RelNode currentRel, - Map map, CorelMap cm) { - this.currentRel = Objects.requireNonNull(currentRel); - this.map = Objects.requireNonNull(map); - this.cm = Objects.requireNonNull(cm); - } - - @Override public RexNode visitFieldAccess(RexFieldAccess fieldAccess) { - int newInputOutputOffset = 0; - for (RelNode input : currentRel.getInputs()) { - final Frame frame = map.get(input); - - if (frame != null) { - // try to find in this input rel the position of corVar - final CorRef corRef = cm.mapFieldAccessToCorRef.get(fieldAccess); - - if (corRef != null) { - Integer newInputPos = frame.corDefOutputs.get(corRef.def()); - if (newInputPos != null) { - // This input does produce the corVar referenced. - return new RexInputRef(newInputPos + newInputOutputOffset, - frame.r.getRowType().getFieldList().get(newInputPos) - .getType()); - } - } - - // this input does not produce the corVar needed - newInputOutputOffset += frame.r.getRowType().getFieldCount(); - } else { - // this input is not rewritten - newInputOutputOffset += input.getRowType().getFieldCount(); - } - } - return fieldAccess; - } - - @Override public RexNode visitInputRef(RexInputRef inputRef) { - final RexInputRef ref = getNewForOldInputRef(currentRel, map, inputRef); - if (ref.getIndex() == inputRef.getIndex() - && ref.getType() == inputRef.getType()) { - return inputRef; // re-use old object, to prevent needless expr cloning - } - return ref; - } - } - - /** Shuttle that removes correlations. */ - private class RemoveCorrelationRexShuttle extends RexShuttle { - final RexBuilder rexBuilder; - final RelDataTypeFactory typeFactory; - final boolean projectPulledAboveLeftCorrelator; - final RexInputRef nullIndicator; - final ImmutableSet isCount; - - RemoveCorrelationRexShuttle( - RexBuilder rexBuilder, - boolean projectPulledAboveLeftCorrelator, - RexInputRef nullIndicator, - Set isCount) { - this.projectPulledAboveLeftCorrelator = - projectPulledAboveLeftCorrelator; - this.nullIndicator = nullIndicator; // may be null - this.isCount = ImmutableSet.copyOf(isCount); - this.rexBuilder = rexBuilder; - this.typeFactory = rexBuilder.getTypeFactory(); - } - - private RexNode createCaseExpression( - RexInputRef nullInputRef, - RexLiteral lit, - RexNode rexNode) { - RexNode[] caseOperands = new RexNode[3]; - - // Construct a CASE expression to handle the null indicator. - // - // This also covers the case where a left correlated sub-query - // projects fields from outer relation. Since LOJ cannot produce - // nulls on the LHS, the projection now need to make a nullable LHS - // reference using a nullability indicator. If this this indicator - // is null, it means the sub-query does not produce any value. As a - // result, any RHS ref by this usbquery needs to produce null value. - - // WHEN indicator IS NULL - caseOperands[0] = - rexBuilder.makeCall( - SqlStdOperatorTable.IS_NULL, - new RexInputRef( - nullInputRef.getIndex(), - typeFactory.createTypeWithNullability( - nullInputRef.getType(), - true))); - - // THEN CAST(NULL AS newInputTypeNullable) - caseOperands[1] = - rexBuilder.makeCast( - typeFactory.createTypeWithNullability( - rexNode.getType(), - true), - lit); - - // ELSE cast (newInput AS newInputTypeNullable) END - caseOperands[2] = - rexBuilder.makeCast( - typeFactory.createTypeWithNullability( - rexNode.getType(), - true), - rexNode); - - return rexBuilder.makeCall( - SqlStdOperatorTable.CASE, - caseOperands); - } - - @Override public RexNode visitFieldAccess(RexFieldAccess fieldAccess) { - if (cm.mapFieldAccessToCorRef.containsKey(fieldAccess)) { - // if it is a corVar, change it to be input ref. - CorRef corVar = cm.mapFieldAccessToCorRef.get(fieldAccess); - - // corVar offset should point to the leftInput of currentRel, - // which is the Correlate. - RexNode newRexNode = - new RexInputRef(corVar.field, fieldAccess.getType()); - - if (projectPulledAboveLeftCorrelator - && (nullIndicator != null)) { - // need to enforce nullability by applying an additional - // cast operator over the transformed expression. - newRexNode = - createCaseExpression( - nullIndicator, - rexBuilder.constantNull(), - newRexNode); - } - return newRexNode; - } - return fieldAccess; - } - - @Override public RexNode visitInputRef(RexInputRef inputRef) { - if (currentRel instanceof LogicalCorrelate) { - // if this rel references corVar - // and now it needs to be rewritten - // it must have been pulled above the Correlate - // replace the input ref to account for the LHS of the - // Correlate - final int leftInputFieldCount = - ((LogicalCorrelate) currentRel).getLeft().getRowType() - .getFieldCount(); - RelDataType newType = inputRef.getType(); - - if (projectPulledAboveLeftCorrelator) { - newType = - typeFactory.createTypeWithNullability(newType, true); - } - - int pos = inputRef.getIndex(); - RexInputRef newInputRef = - new RexInputRef(leftInputFieldCount + pos, newType); - - if ((isCount != null) && isCount.contains(pos)) { - return createCaseExpression( - newInputRef, - rexBuilder.makeExactLiteral(BigDecimal.ZERO), - newInputRef); - } else { - return newInputRef; - } - } - return inputRef; - } - - @Override public RexNode visitLiteral(RexLiteral literal) { - // Use nullIndicator to decide whether to project null. - // Do nothing if the literal is null. - if (!RexUtil.isNull(literal) - && projectPulledAboveLeftCorrelator - && (nullIndicator != null)) { - return createCaseExpression( - nullIndicator, - rexBuilder.constantNull(), - literal); - } - return literal; - } - - @Override public RexNode visitCall(final RexCall call) { - RexNode newCall; - - boolean[] update = {false}; - List clonedOperands = visitList(call.operands, update); - if (update[0]) { - SqlOperator operator = call.getOperator(); - - boolean isSpecialCast = false; - if (operator instanceof SqlFunction) { - SqlFunction function = (SqlFunction) operator; - if (function.getKind() == SqlKind.CAST) { - if (call.operands.size() < 2) { - isSpecialCast = true; - } - } - } - - final RelDataType newType; - if (!isSpecialCast) { - // TODO: ideally this only needs to be called if the result - // type will also change. However, since that requires - // support from type inference rules to tell whether a rule - // decides return type based on input types, for now all - // operators will be recreated with new type if any operand - // changed, unless the operator has "built-in" type. - newType = rexBuilder.deriveReturnType(operator, clonedOperands); - } else { - // Use the current return type when creating a new call, for - // operators with return type built into the operator - // definition, and with no type inference rules, such as - // cast function with less than 2 operands. - - // TODO: Comments in RexShuttle.visitCall() mention other - // types in this category. Need to resolve those together - // and preferably in the base class RexShuttle. - newType = call.getType(); - } - newCall = - rexBuilder.makeCall( - newType, - operator, - clonedOperands); - } else { - newCall = call; - } - - if (projectPulledAboveLeftCorrelator && (nullIndicator != null)) { - return createCaseExpression( - nullIndicator, - rexBuilder.constantNull(), - newCall); - } - return newCall; - } - } - - /** - * Rule to remove single_value rel. For cases like - * - *
AggRel single_value proj/filter/agg/ join on unique LHS key - * AggRel single group
- */ - private final class RemoveSingleAggregateRule extends RelOptRule { - RemoveSingleAggregateRule(RelBuilderFactory relBuilderFactory) { - super( - operand( - LogicalAggregate.class, - operand( - LogicalProject.class, - operand(LogicalAggregate.class, any()))), - relBuilderFactory, null); - } - - public void onMatch(RelOptRuleCall call) { - LogicalAggregate singleAggregate = call.rel(0); - LogicalProject project = call.rel(1); - LogicalAggregate aggregate = call.rel(2); - - // check singleAggRel is single_value agg - if ((!singleAggregate.getGroupSet().isEmpty()) - || (singleAggregate.getAggCallList().size() != 1) - || !(singleAggregate.getAggCallList().get(0).getAggregation() - instanceof SqlSingleValueAggFunction)) { - return; - } - - // check projRel only projects one expression - // check this project only projects one expression, i.e. scalar - // sub-queries. - List projExprs = project.getProjects(); - if (projExprs.size() != 1) { - return; - } - - // check the input to project is an aggregate on the entire input - if (!aggregate.getGroupSet().isEmpty()) { - return; - } - - // singleAggRel produces a nullable type, so create the new - // projection that casts proj expr to a nullable type. - final RelBuilder relBuilder = call.builder(); - final RelDataType type = - relBuilder.getTypeFactory() - .createTypeWithNullability(projExprs.get(0).getType(), true); - final RexNode cast = - relBuilder.getRexBuilder().makeCast(type, projExprs.get(0)); - relBuilder.push(aggregate) - .project(cast); - call.transformTo(relBuilder.build()); - } - } - - /** Planner rule that removes correlations for scalar projects. */ - private final class RemoveCorrelationForScalarProjectRule extends RelOptRule { - RemoveCorrelationForScalarProjectRule(RelBuilderFactory relBuilderFactory) { - super( - operand(LogicalCorrelate.class, - operand(RelNode.class, any()), - operand(LogicalAggregate.class, - operand(LogicalProject.class, - operand(RelNode.class, any())))), - relBuilderFactory, null); - } - - public void onMatch(RelOptRuleCall call) { - final LogicalCorrelate correlate = call.rel(0); - final RelNode left = call.rel(1); - final LogicalAggregate aggregate = call.rel(2); - final LogicalProject project = call.rel(3); - RelNode right = call.rel(4); - final RelOptCluster cluster = correlate.getCluster(); - - setCurrent(call.getPlanner().getRoot(), correlate); - - // Check for this pattern. - // The pattern matching could be simplified if rules can be applied - // during decorrelation. - // - // Correlate(left correlation, condition = true) - // leftInput - // Aggregate (groupby (0) single_value()) - // Project-A (may reference corVar) - // rightInput - final JoinRelType joinType = correlate.getJoinType(); - - // corRel.getCondition was here, however Correlate was updated so it - // never includes a join condition. The code was not modified for brevity. - RexNode joinCond = relBuilder.literal(true); - if ((joinType != JoinRelType.LEFT) - || (joinCond != relBuilder.literal(true))) { - return; - } - - // check that the agg is of the following type: - // doing a single_value() on the entire input - if ((!aggregate.getGroupSet().isEmpty()) - || (aggregate.getAggCallList().size() != 1) - || !(aggregate.getAggCallList().get(0).getAggregation() - instanceof SqlSingleValueAggFunction)) { - return; - } - - // check this project only projects one expression, i.e. scalar - // sub-queries. - if (project.getProjects().size() != 1) { - return; - } - - int nullIndicatorPos; - - if ((right instanceof LogicalFilter) - && cm.mapRefRelToCorRef.containsKey(right)) { - // rightInput has this shape: - // - // Filter (references corVar) - // filterInput - - // If rightInput is a filter and contains correlated - // reference, make sure the correlated keys in the filter - // condition forms a unique key of the RHS. - - LogicalFilter filter = (LogicalFilter) right; - right = filter.getInput(); - - assert right instanceof HepRelVertex; - right = ((HepRelVertex) right).getCurrentRel(); - - // check filter input contains no correlation - if (RelOptUtil.getVariablesUsed(right).size() > 0) { - return; - } - - // extract the correlation out of the filter - - // First breaking up the filter conditions into equality - // comparisons between rightJoinKeys (from the original - // filterInput) and correlatedJoinKeys. correlatedJoinKeys - // can be expressions, while rightJoinKeys need to be input - // refs. These comparisons are AND'ed together. - List tmpRightJoinKeys = new ArrayList<>(); - List correlatedJoinKeys = new ArrayList<>(); - RelOptUtil.splitCorrelatedFilterCondition( - filter, - tmpRightJoinKeys, - correlatedJoinKeys, - false); - - // check that the columns referenced in these comparisons form - // an unique key of the filterInput - final List rightJoinKeys = new ArrayList<>(); - for (RexNode key : tmpRightJoinKeys) { - assert key instanceof RexInputRef; - rightJoinKeys.add((RexInputRef) key); - } - - // check that the columns referenced in rightJoinKeys form an - // unique key of the filterInput - if (rightJoinKeys.isEmpty()) { - return; - } - - // The join filters out the nulls. So, it's ok if there are - // nulls in the join keys. - final RelMetadataQuery mq = call.getMetadataQuery(); - if (!RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(mq, right, - rightJoinKeys)) { - SQL2REL_LOGGER.debug("{} are not unique keys for {}", - rightJoinKeys.toString(), right.toString()); - return; - } - - RexUtil.FieldAccessFinder visitor = - new RexUtil.FieldAccessFinder(); - RexUtil.apply(visitor, correlatedJoinKeys, null); - List correlatedKeyList = - visitor.getFieldAccessList(); - - if (!checkCorVars(correlate, project, filter, correlatedKeyList)) { - return; - } - - // Change the plan to this structure. - // Note that the Aggregate is removed. - // - // Project-A' (replace corVar to input ref from the Join) - // Join (replace corVar to input ref from leftInput) - // leftInput - // rightInput (previously filterInput) - - // Change the filter condition into a join condition - joinCond = - removeCorrelationExpr(filter.getCondition(), false); - - nullIndicatorPos = - left.getRowType().getFieldCount() - + rightJoinKeys.get(0).getIndex(); - } else if (cm.mapRefRelToCorRef.containsKey(project)) { - // check filter input contains no correlation - if (RelOptUtil.getVariablesUsed(right).size() > 0) { - return; - } - - if (!checkCorVars(correlate, project, null, null)) { - return; - } - - // Change the plan to this structure. - // - // Project-A' (replace corVar to input ref from Join) - // Join (left, condition = true) - // leftInput - // Aggregate(groupby(0), single_value(0), s_v(1)....) - // Project-B (everything from input plus literal true) - // projectInput - - // make the new Project to provide a null indicator - right = - createProjectWithAdditionalExprs(right, - ImmutableList.of( - Pair.of(relBuilder.literal(true), "nullIndicator"))); - - // make the new aggRel - right = - RelOptUtil.createSingleValueAggRel(cluster, right); - - // The last field: - // single_value(true) - // is the nullIndicator - nullIndicatorPos = - left.getRowType().getFieldCount() - + right.getRowType().getFieldCount() - 1; - } else { - return; - } - - // make the new join rel - LogicalJoin join = - LogicalJoin.create(left, right, joinCond, - ImmutableSet.of(), joinType); - - RelNode newProject = - projectJoinOutputWithNullability(join, project, nullIndicatorPos); - - call.transformTo(newProject); - - removeCorVarFromTree(correlate); - } - } - - /** Planner rule that removes correlations for scalar aggregates. */ - private final class RemoveCorrelationForScalarAggregateRule - extends RelOptRule { - RemoveCorrelationForScalarAggregateRule(RelBuilderFactory relBuilderFactory) { - super( - operand(LogicalCorrelate.class, - operand(RelNode.class, any()), - operand(LogicalProject.class, - operandJ(LogicalAggregate.class, null, Aggregate::isSimple, - operand(LogicalProject.class, - operand(RelNode.class, any()))))), - relBuilderFactory, null); - } - - public void onMatch(RelOptRuleCall call) { - final LogicalCorrelate correlate = call.rel(0); - final RelNode left = call.rel(1); - final LogicalProject aggOutputProject = call.rel(2); - final LogicalAggregate aggregate = call.rel(3); - final LogicalProject aggInputProject = call.rel(4); - RelNode right = call.rel(5); - final RelBuilder builder = call.builder(); - final RexBuilder rexBuilder = builder.getRexBuilder(); - final RelOptCluster cluster = correlate.getCluster(); - - setCurrent(call.getPlanner().getRoot(), correlate); - - // check for this pattern - // The pattern matching could be simplified if rules can be applied - // during decorrelation, - // - // CorrelateRel(left correlation, condition = true) - // leftInput - // Project-A (a RexNode) - // Aggregate (groupby (0), agg0(), agg1()...) - // Project-B (references coVar) - // rightInput - - // check aggOutputProject projects only one expression - final List aggOutputProjects = aggOutputProject.getProjects(); - if (aggOutputProjects.size() != 1) { - return; - } - - final JoinRelType joinType = correlate.getJoinType(); - // corRel.getCondition was here, however Correlate was updated so it - // never includes a join condition. The code was not modified for brevity. - RexNode joinCond = rexBuilder.makeLiteral(true); - if ((joinType != JoinRelType.LEFT) - || (joinCond != rexBuilder.makeLiteral(true))) { - return; - } - - // check that the agg is on the entire input - if (!aggregate.getGroupSet().isEmpty()) { - return; - } - - final List aggInputProjects = aggInputProject.getProjects(); - - final List aggCalls = aggregate.getAggCallList(); - final Set isCountStar = new HashSet<>(); - - // mark if agg produces count(*) which needs to reference the - // nullIndicator after the transformation. - int k = -1; - for (AggregateCall aggCall : aggCalls) { - ++k; - if ((aggCall.getAggregation() instanceof SqlCountAggFunction) - && (aggCall.getArgList().size() == 0)) { - isCountStar.add(k); - } - } - - if ((right instanceof LogicalFilter) - && cm.mapRefRelToCorRef.containsKey(right)) { - // rightInput has this shape: - // - // Filter (references corVar) - // filterInput - LogicalFilter filter = (LogicalFilter) right; - right = filter.getInput(); - - assert right instanceof HepRelVertex; - right = ((HepRelVertex) right).getCurrentRel(); - - // check filter input contains no correlation - if (RelOptUtil.getVariablesUsed(right).size() > 0) { - return; - } - - // check filter condition type First extract the correlation out - // of the filter - - // First breaking up the filter conditions into equality - // comparisons between rightJoinKeys(from the original - // filterInput) and correlatedJoinKeys. correlatedJoinKeys - // can only be RexFieldAccess, while rightJoinKeys can be - // expressions. These comparisons are AND'ed together. - List rightJoinKeys = new ArrayList<>(); - List tmpCorrelatedJoinKeys = new ArrayList<>(); - RelOptUtil.splitCorrelatedFilterCondition( - filter, - rightJoinKeys, - tmpCorrelatedJoinKeys, - true); - - // make sure the correlated reference forms a unique key check - // that the columns referenced in these comparisons form an - // unique key of the leftInput - List correlatedJoinKeys = new ArrayList<>(); - List correlatedInputRefJoinKeys = new ArrayList<>(); - for (RexNode joinKey : tmpCorrelatedJoinKeys) { - assert joinKey instanceof RexFieldAccess; - correlatedJoinKeys.add((RexFieldAccess) joinKey); - RexNode correlatedInputRef = - removeCorrelationExpr(joinKey, false); - assert correlatedInputRef instanceof RexInputRef; - correlatedInputRefJoinKeys.add( - (RexInputRef) correlatedInputRef); - } - - // check that the columns referenced in rightJoinKeys form an - // unique key of the filterInput - if (correlatedInputRefJoinKeys.isEmpty()) { - return; - } - - // The join filters out the nulls. So, it's ok if there are - // nulls in the join keys. - final RelMetadataQuery mq = call.getMetadataQuery(); - if (!RelMdUtil.areColumnsDefinitelyUniqueWhenNullsFiltered(mq, left, - correlatedInputRefJoinKeys)) { - SQL2REL_LOGGER.debug("{} are not unique keys for {}", - correlatedJoinKeys.toString(), left.toString()); - return; - } - - // check corVar references are valid - if (!checkCorVars(correlate, - aggInputProject, - filter, - correlatedJoinKeys)) { - return; - } - - // Rewrite the above plan: - // - // Correlate(left correlation, condition = true) - // leftInput - // Project-A (a RexNode) - // Aggregate (groupby(0), agg0(),agg1()...) - // Project-B (may reference corVar) - // Filter (references corVar) - // rightInput (no correlated reference) - // - - // to this plan: - // - // Project-A' (all gby keys + rewritten nullable ProjExpr) - // Aggregate (groupby(all left input refs) - // agg0(rewritten expression), - // agg1()...) - // Project-B' (rewritten original projected exprs) - // Join(replace corVar w/ input ref from leftInput) - // leftInput - // rightInput - // - - // In the case where agg is count(*) or count($corVar), it is - // changed to count(nullIndicator). - // Note: any non-nullable field from the RHS can be used as - // the indicator however a "true" field is added to the - // projection list from the RHS for simplicity to avoid - // searching for non-null fields. - // - // Project-A' (all gby keys + rewritten nullable ProjExpr) - // Aggregate (groupby(all left input refs), - // count(nullIndicator), other aggs...) - // Project-B' (all left input refs plus - // the rewritten original projected exprs) - // Join(replace corVar to input ref from leftInput) - // leftInput - // Project (everything from rightInput plus - // the nullIndicator "true") - // rightInput - // - - // first change the filter condition into a join condition - joinCond = - removeCorrelationExpr(filter.getCondition(), false); - } else if (cm.mapRefRelToCorRef.containsKey(aggInputProject)) { - // check rightInput contains no correlation - if (RelOptUtil.getVariablesUsed(right).size() > 0) { - return; - } - - // check corVar references are valid - if (!checkCorVars(correlate, aggInputProject, null, null)) { - return; - } - - int nFields = left.getRowType().getFieldCount(); - ImmutableBitSet allCols = ImmutableBitSet.range(nFields); - - // leftInput contains unique keys - // i.e. each row is distinct and can group by on all the left - // fields - final RelMetadataQuery mq = call.getMetadataQuery(); - if (!RelMdUtil.areColumnsDefinitelyUnique(mq, left, allCols)) { - SQL2REL_LOGGER.debug("There are no unique keys for {}", left); - return; - } - // - // Rewrite the above plan: - // - // CorrelateRel(left correlation, condition = true) - // leftInput - // Project-A (a RexNode) - // Aggregate (groupby(0), agg0(), agg1()...) - // Project-B (references coVar) - // rightInput (no correlated reference) - // - - // to this plan: - // - // Project-A' (all gby keys + rewritten nullable ProjExpr) - // Aggregate (groupby(all left input refs) - // agg0(rewritten expression), - // agg1()...) - // Project-B' (rewritten original projected exprs) - // Join (LOJ cond = true) - // leftInput - // rightInput - // - - // In the case where agg is count($corVar), it is changed to - // count(nullIndicator). - // Note: any non-nullable field from the RHS can be used as - // the indicator however a "true" field is added to the - // projection list from the RHS for simplicity to avoid - // searching for non-null fields. - // - // Project-A' (all gby keys + rewritten nullable ProjExpr) - // Aggregate (groupby(all left input refs), - // count(nullIndicator), other aggs...) - // Project-B' (all left input refs plus - // the rewritten original projected exprs) - // Join (replace corVar to input ref from leftInput) - // leftInput - // Project (everything from rightInput plus - // the nullIndicator "true") - // rightInput - } else { - return; - } - - RelDataType leftInputFieldType = left.getRowType(); - int leftInputFieldCount = leftInputFieldType.getFieldCount(); - int joinOutputProjExprCount = - leftInputFieldCount + aggInputProjects.size() + 1; - - right = - createProjectWithAdditionalExprs(right, - ImmutableList.of( - Pair.of(rexBuilder.makeLiteral(true), - "nullIndicator"))); - - LogicalJoin join = - LogicalJoin.create(left, right, joinCond, - ImmutableSet.of(), joinType); - - // To the consumer of joinOutputProjRel, nullIndicator is located - // at the end - int nullIndicatorPos = join.getRowType().getFieldCount() - 1; - - RexInputRef nullIndicator = - new RexInputRef( - nullIndicatorPos, - cluster.getTypeFactory().createTypeWithNullability( - join.getRowType().getFieldList() - .get(nullIndicatorPos).getType(), - true)); - - // first project all group-by keys plus the transformed agg input - List joinOutputProjects = new ArrayList<>(); - - // LOJ Join preserves LHS types - for (int i = 0; i < leftInputFieldCount; i++) { - joinOutputProjects.add( - rexBuilder.makeInputRef( - leftInputFieldType.getFieldList().get(i).getType(), i)); - } - - for (RexNode aggInputProjExpr : aggInputProjects) { - joinOutputProjects.add( - removeCorrelationExpr(aggInputProjExpr, - joinType.generatesNullsOnRight(), - nullIndicator)); - } - - joinOutputProjects.add( - rexBuilder.makeInputRef(join, nullIndicatorPos)); - - final RelNode joinOutputProject = builder.push(join) - .project(joinOutputProjects) - .build(); - - // nullIndicator is now at a different location in the output of - // the join - nullIndicatorPos = joinOutputProjExprCount - 1; - - final int groupCount = leftInputFieldCount; - - List newAggCalls = new ArrayList<>(); - k = -1; - for (AggregateCall aggCall : aggCalls) { - ++k; - final List argList; - - if (isCountStar.contains(k)) { - // this is a count(*), transform it to count(nullIndicator) - // the null indicator is located at the end - argList = Collections.singletonList(nullIndicatorPos); - } else { - argList = new ArrayList<>(); - - for (int aggArg : aggCall.getArgList()) { - argList.add(aggArg + groupCount); - } - } - - int filterArg = aggCall.filterArg < 0 ? aggCall.filterArg - : aggCall.filterArg + groupCount; - newAggCalls.add( - aggCall.adaptTo(joinOutputProject, argList, filterArg, - aggregate.getGroupCount(), groupCount)); - } - - ImmutableBitSet groupSet = - ImmutableBitSet.range(groupCount); - LogicalAggregate newAggregate = - LogicalAggregate.create(joinOutputProject, groupSet, null, - newAggCalls); - List newAggOutputProjectList = new ArrayList<>(); - for (int i : groupSet) { - newAggOutputProjectList.add( - rexBuilder.makeInputRef(newAggregate, i)); - } - - RexNode newAggOutputProjects = - removeCorrelationExpr(aggOutputProjects.get(0), false); - newAggOutputProjectList.add( - rexBuilder.makeCast( - cluster.getTypeFactory().createTypeWithNullability( - newAggOutputProjects.getType(), - true), - newAggOutputProjects)); - - builder.push(newAggregate) - .project(newAggOutputProjectList); - call.transformTo(builder.build()); - - removeCorVarFromTree(correlate); - } - } - - // REVIEW jhyde 29-Oct-2007: This rule is non-static, depends on the state - // of members in RelDecorrelator, and has side-effects in the decorrelator. - // This breaks the contract of a planner rule, and the rule will not be - // reusable in other planners. - - // REVIEW jvs 29-Oct-2007: Shouldn't it also be incorporating - // the flavor attribute into the description? - - /** Planner rule that adjusts projects when counts are added. */ - private final class AdjustProjectForCountAggregateRule extends RelOptRule { - final boolean flavor; - - AdjustProjectForCountAggregateRule(boolean flavor, - RelBuilderFactory relBuilderFactory) { - super( - flavor - ? operand(LogicalCorrelate.class, - operand(RelNode.class, any()), - operand(LogicalProject.class, - operand(LogicalAggregate.class, any()))) - : operand(LogicalCorrelate.class, - operand(RelNode.class, any()), - operand(LogicalAggregate.class, any())), - relBuilderFactory, null); - this.flavor = flavor; - } - - public void onMatch(RelOptRuleCall call) { - final LogicalCorrelate correlate = call.rel(0); - final RelNode left = call.rel(1); - final LogicalProject aggOutputProject; - final LogicalAggregate aggregate; - if (flavor) { - aggOutputProject = call.rel(2); - aggregate = call.rel(3); - } else { - aggregate = call.rel(2); - - // Create identity projection - final List> projects = new ArrayList<>(); - final List fields = - aggregate.getRowType().getFieldList(); - for (int i = 0; i < fields.size(); i++) { - projects.add(RexInputRef.of2(projects.size(), fields)); - } - final RelBuilder relBuilder = call.builder(); - relBuilder.push(aggregate) - .projectNamed(Pair.left(projects), Pair.right(projects), true); - aggOutputProject = (LogicalProject) relBuilder.build(); - } - onMatch2(call, correlate, left, aggOutputProject, aggregate); - } - - private void onMatch2( - RelOptRuleCall call, - LogicalCorrelate correlate, - RelNode leftInput, - LogicalProject aggOutputProject, - LogicalAggregate aggregate) { - if (generatedCorRels.contains(correlate)) { - // This Correlate was generated by a previous invocation of - // this rule. No further work to do. - return; - } - - setCurrent(call.getPlanner().getRoot(), correlate); - - // check for this pattern - // The pattern matching could be simplified if rules can be applied - // during decorrelation, - // - // CorrelateRel(left correlation, condition = true) - // leftInput - // Project-A (a RexNode) - // Aggregate (groupby (0), agg0(), agg1()...) - - // check aggOutputProj projects only one expression - List aggOutputProjExprs = aggOutputProject.getProjects(); - if (aggOutputProjExprs.size() != 1) { - return; - } - - JoinRelType joinType = correlate.getJoinType(); - // corRel.getCondition was here, however Correlate was updated so it - // never includes a join condition. The code was not modified for brevity. - RexNode joinCond = relBuilder.literal(true); - if ((joinType != JoinRelType.LEFT) - || (joinCond != relBuilder.literal(true))) { - return; - } - - // check that the agg is on the entire input - if (!aggregate.getGroupSet().isEmpty()) { - return; - } - - List aggCalls = aggregate.getAggCallList(); - Set isCount = new HashSet<>(); - - // remember the count() positions - int i = -1; - for (AggregateCall aggCall : aggCalls) { - ++i; - if (aggCall.getAggregation() instanceof SqlCountAggFunction) { - isCount.add(i); - } - } - - // now rewrite the plan to - // - // Project-A' (all LHS plus transformed original projections, - // replacing references to count() with case statement) - // Correlate(left correlation, condition = true) - // leftInput - // Aggregate(groupby (0), agg0(), agg1()...) - // - LogicalCorrelate newCorrelate = - LogicalCorrelate.create(leftInput, aggregate, - correlate.getCorrelationId(), correlate.getRequiredColumns(), - correlate.getJoinType()); - - // remember this rel so we don't fire rule on it again - // REVIEW jhyde 29-Oct-2007: rules should not save state; rule - // should recognize patterns where it does or does not need to do - // work - generatedCorRels.add(newCorrelate); - - // need to update the mapCorToCorRel Update the output position - // for the corVars: only pass on the corVars that are not used in - // the join key. - if (cm.mapCorToCorRel.get(correlate.getCorrelationId()) == correlate) { - cm.mapCorToCorRel.put(correlate.getCorrelationId(), newCorrelate); - } - - RelNode newOutput = - aggregateCorrelatorOutput(newCorrelate, aggOutputProject, isCount); - - call.transformTo(newOutput); - } - } - - /** - * A unique reference to a correlation field. - * - *

For instance, if a RelNode references emp.name multiple times, it would - * result in multiple {@code CorRef} objects that differ just in - * {@link CorRef#uniqueKey}. - */ - static class CorRef implements Comparable { - public final int uniqueKey; - public final CorrelationId corr; - public final int field; - - CorRef(CorrelationId corr, int field, int uniqueKey) { - this.corr = corr; - this.field = field; - this.uniqueKey = uniqueKey; - } - - @Override public String toString() { - return corr.getName() + '.' + field; - } - - @Override public int hashCode() { - return Objects.hash(uniqueKey, corr, field); - } - - @Override public boolean equals(Object o) { - return this == o - || o instanceof CorRef - && uniqueKey == ((CorRef) o).uniqueKey - && corr == ((CorRef) o).corr - && field == ((CorRef) o).field; - } - - public int compareTo(@Nonnull CorRef o) { - int c = corr.compareTo(o.corr); - if (c != 0) { - return c; - } - c = Integer.compare(field, o.field); - if (c != 0) { - return c; - } - return Integer.compare(uniqueKey, o.uniqueKey); - } - - public CorDef def() { - return new CorDef(corr, field); - } - } - - /** A correlation and a field. */ - static class CorDef implements Comparable { - public final CorrelationId corr; - public final int field; - - CorDef(CorrelationId corr, int field) { - this.corr = corr; - this.field = field; - } - - @Override public String toString() { - return corr.getName() + '.' + field; - } - - @Override public int hashCode() { - return Objects.hash(corr, field); - } - - @Override public boolean equals(Object o) { - return this == o - || o instanceof CorDef - && corr == ((CorDef) o).corr - && field == ((CorDef) o).field; - } - - public int compareTo(@Nonnull CorDef o) { - int c = corr.compareTo(o.corr); - if (c != 0) { - return c; - } - return Integer.compare(field, o.field); - } - } - - /** A map of the locations of - * {@link org.apache.calcite.rel.logical.LogicalCorrelate} - * in a tree of {@link RelNode}s. - * - *

It is used to drive the decorrelation process. - * Treat it as immutable; rebuild if you modify the tree. - * - *

There are three maps:

    - * - *
  1. {@link #mapRefRelToCorRef} maps a {@link RelNode} to the correlated - * variables it references; - * - *
  2. {@link #mapCorToCorRel} maps a correlated variable to the - * {@link Correlate} providing it; - * - *
  3. {@link #mapFieldAccessToCorRef} maps a rex field access to - * the corVar it represents. Because typeFlattener does not clone or - * modify a correlated field access this map does not need to be - * updated. - * - *
*/ - private static class CorelMap { - private final Multimap mapRefRelToCorRef; - private final SortedMap mapCorToCorRel; - private final Map mapFieldAccessToCorRef; - - // TODO: create immutable copies of all maps - private CorelMap(Multimap mapRefRelToCorRef, - SortedMap mapCorToCorRel, - Map mapFieldAccessToCorRef) { - this.mapRefRelToCorRef = mapRefRelToCorRef; - this.mapCorToCorRel = mapCorToCorRel; - this.mapFieldAccessToCorRef = ImmutableMap.copyOf(mapFieldAccessToCorRef); - } - - @Override public String toString() { - return "mapRefRelToCorRef=" + mapRefRelToCorRef - + "\nmapCorToCorRel=" + mapCorToCorRel - + "\nmapFieldAccessToCorRef=" + mapFieldAccessToCorRef - + "\n"; - } - - @Override public boolean equals(Object obj) { - return obj == this - || obj instanceof CorelMap - && mapRefRelToCorRef.equals(((CorelMap) obj).mapRefRelToCorRef) - && mapCorToCorRel.equals(((CorelMap) obj).mapCorToCorRel) - && mapFieldAccessToCorRef.equals( - ((CorelMap) obj).mapFieldAccessToCorRef); - } - - @Override public int hashCode() { - return Objects.hash(mapRefRelToCorRef, mapCorToCorRel, - mapFieldAccessToCorRef); - } - - /** Creates a CorelMap with given contents. */ - public static CorelMap of( - SortedSetMultimap mapRefRelToCorVar, - SortedMap mapCorToCorRel, - Map mapFieldAccessToCorVar) { - return new CorelMap(mapRefRelToCorVar, mapCorToCorRel, - mapFieldAccessToCorVar); - } - - /** - * Returns whether there are any correlating variables in this statement. - * - * @return whether there are any correlating variables - */ - public boolean hasCorrelation() { - return !mapCorToCorRel.isEmpty(); - } - } - - /** Builds a {@link org.apache.calcite.sql2rel.RelDecorrelator.CorelMap}. */ - private static class CorelMapBuilder extends RelShuttleImpl { - final SortedMap mapCorToCorRel = - new TreeMap<>(); - - final SortedSetMultimap mapRefRelToCorRef = - MultimapBuilder.SortedSetMultimapBuilder.hashKeys() - .treeSetValues() - .build(); - - final Map mapFieldAccessToCorVar = new HashMap<>(); - - final Holder offset = Holder.of(0); - int corrIdGenerator = 0; - - /** Creates a CorelMap by iterating over a {@link RelNode} tree. */ - CorelMap build(RelNode... rels) { - for (RelNode rel : rels) { - stripHep(rel).accept(this); - } - return new CorelMap(mapRefRelToCorRef, mapCorToCorRel, - mapFieldAccessToCorVar); - } - - @Override public RelNode visit(LogicalJoin join) { - try { - stack.push(join); - join.getCondition().accept(rexVisitor(join)); - } finally { - stack.pop(); - } - return visitJoin(join); - } - - @Override protected RelNode visitChild(RelNode parent, int i, - RelNode input) { - return super.visitChild(parent, i, stripHep(input)); - } - - @Override public RelNode visit(LogicalCorrelate correlate) { - mapCorToCorRel.put(correlate.getCorrelationId(), correlate); - return visitJoin(correlate); - } - - private RelNode visitJoin(BiRel join) { - final int x = offset.get(); - visitChild(join, 0, join.getLeft()); - offset.set(x + join.getLeft().getRowType().getFieldCount()); - visitChild(join, 1, join.getRight()); - offset.set(x); - return join; - } - - @Override public RelNode visit(final LogicalFilter filter) { - try { - stack.push(filter); - filter.getCondition().accept(rexVisitor(filter)); - } finally { - stack.pop(); - } - return super.visit(filter); - } - - @Override public RelNode visit(LogicalProject project) { - try { - stack.push(project); - for (RexNode node : project.getProjects()) { - node.accept(rexVisitor(project)); - } - } finally { - stack.pop(); - } - return super.visit(project); - } - - private RexVisitorImpl rexVisitor(final RelNode rel) { - return new RexVisitorImpl(true) { - @Override public Void visitFieldAccess(RexFieldAccess fieldAccess) { - final RexNode ref = fieldAccess.getReferenceExpr(); - if (ref instanceof RexCorrelVariable) { - final RexCorrelVariable var = (RexCorrelVariable) ref; - if (mapFieldAccessToCorVar.containsKey(fieldAccess)) { - // for cases where different Rel nodes are referring to - // same correlation var (e.g. in case of NOT IN) - // avoid generating another correlation var - // and record the 'rel' is using the same correlation - mapRefRelToCorRef.put(rel, - mapFieldAccessToCorVar.get(fieldAccess)); - } else { - final CorRef correlation = - new CorRef(var.id, fieldAccess.getField().getIndex(), - corrIdGenerator++); - mapFieldAccessToCorVar.put(fieldAccess, correlation); - mapRefRelToCorRef.put(rel, correlation); - } - } - return super.visitFieldAccess(fieldAccess); - } - - @Override public Void visitSubQuery(RexSubQuery subQuery) { - subQuery.rel.accept(CorelMapBuilder.this); - return super.visitSubQuery(subQuery); - } - }; - } - } - - /** Frame describing the relational expression after decorrelation - * and where to find the output fields and correlation variables - * among its output fields. */ - static class Frame { - final RelNode r; - final ImmutableSortedMap corDefOutputs; - final ImmutableSortedMap oldToNewOutputs; - - Frame(RelNode oldRel, RelNode r, SortedMap corDefOutputs, - Map oldToNewOutputs) { - this.r = Objects.requireNonNull(r); - this.corDefOutputs = ImmutableSortedMap.copyOf(corDefOutputs); - this.oldToNewOutputs = ImmutableSortedMap.copyOf(oldToNewOutputs); - assert allLessThan(this.corDefOutputs.values(), - r.getRowType().getFieldCount(), Litmus.THROW); - assert allLessThan(this.oldToNewOutputs.keySet(), - oldRel.getRowType().getFieldCount(), Litmus.THROW); - assert allLessThan(this.oldToNewOutputs.values(), - r.getRowType().getFieldCount(), Litmus.THROW); - } - } -} - -// End RelDecorrelator.java diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java deleted file mode 100644 index 23ec403a11..0000000000 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java +++ /dev/null @@ -1,363 +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.planner.plan.rules.logical; - -import org.apache.calcite.plan.RelOptRule; -import org.apache.calcite.plan.RelOptRuleCall; -import org.apache.calcite.plan.RelOptRuleOperand; -import org.apache.calcite.plan.RelOptUtil; -import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.core.EquiJoin; -import org.apache.calcite.rel.core.Filter; -import org.apache.calcite.rel.core.Join; -import org.apache.calcite.rel.core.JoinRelType; -import org.apache.calcite.rel.core.RelFactories; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexNode; -import org.apache.calcite.rex.RexUtil; -import org.apache.calcite.tools.RelBuilder; -import org.apache.calcite.tools.RelBuilderFactory; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Objects; - -/** - * This rules is copied from Calcite's {@link org.apache.calcite.rel.rules.FilterJoinRule}. - * NOTES: This file should be deleted when CALCITE-3170 is fixed, - * and please make sure to synchronize with FlinkFilterJoinRule in flink planner when changing this class. - * Modification: - * - Handles the ON condition of anti-join can not be pushed down, lines added (192-198) - */ - -/** - * Planner rule that pushes filters above and - * within a join node into the join node and/or its children nodes. - */ -public abstract class FlinkFilterJoinRule extends RelOptRule { - /** Predicate that always returns true. With this predicate, every filter - * will be pushed into the ON clause. */ - public static final Predicate TRUE_PREDICATE = (join, joinType, exp) -> true; - - /** Rule that pushes predicates from a Filter into the Join below them. */ - public static final FlinkFilterJoinRule FILTER_ON_JOIN = - new FlinkFilterIntoJoinRule(true, RelFactories.LOGICAL_BUILDER, - TRUE_PREDICATE); - - /** Dumber version of {@link #FILTER_ON_JOIN}. Not intended for production - * use, but keeps some tests working for which {@code FILTER_ON_JOIN} is too - * smart. */ - public static final FlinkFilterJoinRule DUMB_FILTER_ON_JOIN = - new FlinkFilterIntoJoinRule(false, RelFactories.LOGICAL_BUILDER, - TRUE_PREDICATE); - - /** Rule that pushes predicates in a Join into the inputs to the join. */ - public static final FlinkFilterJoinRule JOIN = - new FlinkJoinConditionPushRule(RelFactories.LOGICAL_BUILDER, TRUE_PREDICATE); - - /** Whether to try to strengthen join-type. */ - private final boolean smart; - - /** Predicate that returns whether a filter is valid in the ON clause of a - * join for this particular kind of join. If not, Calcite will push it back to - * above the join. */ - private final Predicate predicate; - - //~ Constructors ----------------------------------------------------------- - - /** - * Creates a FilterProjectTransposeRule with an explicit root operand and - * factories. - */ - protected FlinkFilterJoinRule(RelOptRuleOperand operand, String id, - boolean smart, RelBuilderFactory relBuilderFactory, Predicate predicate) { - super(operand, relBuilderFactory, "FlinkFilterJoinRule:" + id); - this.smart = smart; - this.predicate = Objects.requireNonNull(predicate); - } - - /** - * Creates a FlinkFilterJoinRule with an explicit root operand and - * factories. - */ - @Deprecated // to be removed before 2.0 - protected FlinkFilterJoinRule(RelOptRuleOperand operand, String id, - boolean smart, RelFactories.FilterFactory filterFactory, - RelFactories.ProjectFactory projectFactory) { - this(operand, id, smart, RelBuilder.proto(filterFactory, projectFactory), - TRUE_PREDICATE); - } - - /** - * Creates a FilterProjectTransposeRule with an explicit root operand and - * factories. - */ - @Deprecated // to be removed before 2.0 - protected FlinkFilterJoinRule(RelOptRuleOperand operand, String id, - boolean smart, RelFactories.FilterFactory filterFactory, - RelFactories.ProjectFactory projectFactory, - Predicate predicate) { - this(operand, id, smart, RelBuilder.proto(filterFactory, projectFactory), - predicate); - } - - //~ Methods ---------------------------------------------------------------- - - protected void perform(RelOptRuleCall call, Filter filter, - Join join) { - final List joinFilters = - RelOptUtil.conjunctions(join.getCondition()); - final List origJoinFilters = com.google.common.collect.ImmutableList.copyOf(joinFilters); - - // If there is only the joinRel, - // make sure it does not match a cartesian product joinRel - // (with "true" condition), otherwise this rule will be applied - // again on the new cartesian product joinRel. - if (filter == null && joinFilters.isEmpty()) { - return; - } - - final List aboveFilters = - filter != null - ? RelOptUtil.conjunctions(filter.getCondition()) - : new ArrayList<>(); - final com.google.common.collect.ImmutableList origAboveFilters = - com.google.common.collect.ImmutableList.copyOf(aboveFilters); - - // Simplify Outer Joins - JoinRelType joinType = join.getJoinType(); - if (smart - && !origAboveFilters.isEmpty() - && join.getJoinType() != JoinRelType.INNER) { - joinType = RelOptUtil.simplifyJoin(join, origAboveFilters, joinType); - } - - final List leftFilters = new ArrayList<>(); - final List rightFilters = new ArrayList<>(); - - // TODO - add logic to derive additional filters. E.g., from - // (t1.a = 1 AND t2.a = 2) OR (t1.b = 3 AND t2.b = 4), you can - // derive table filters: - // (t1.a = 1 OR t1.b = 3) - // (t2.a = 2 OR t2.b = 4) - - // Try to push down above filters. These are typically where clause - // filters. They can be pushed down if they are not on the NULL - // generating side. - boolean filterPushed = false; - if (RelOptUtil.classifyFilters( - join, - aboveFilters, - joinType, - !(join instanceof EquiJoin), - !joinType.generatesNullsOnLeft(), - !joinType.generatesNullsOnRight(), - joinFilters, - leftFilters, - rightFilters)) { - filterPushed = true; - } - - // Move join filters up if needed - validateJoinFilters(aboveFilters, joinFilters, join, joinType); - - // If no filter got pushed after validate, reset filterPushed flag - if (leftFilters.isEmpty() - && rightFilters.isEmpty() - && joinFilters.size() == origJoinFilters.size()) { - if (com.google.common.collect.Sets.newHashSet(joinFilters) - .equals(com.google.common.collect.Sets.newHashSet(origJoinFilters))) { - filterPushed = false; - } - } - - boolean isAntiJoin = joinType == JoinRelType.ANTI; - - // Try to push down filters in ON clause. A ON clause filter can only be - // pushed down if it does not affect the non-matching set, i.e. it is - // not on the side which is preserved. - // A ON clause filter of anti-join can not be pushed down. - if (!isAntiJoin && RelOptUtil.classifyFilters( - join, - joinFilters, - joinType, - false, - !joinType.generatesNullsOnRight(), - !joinType.generatesNullsOnLeft(), - joinFilters, - leftFilters, - rightFilters)) { - filterPushed = true; - } - - // if nothing actually got pushed and there is nothing leftover, - // then this rule is a no-op - if ((!filterPushed - && joinType == join.getJoinType()) - || (joinFilters.isEmpty() - && leftFilters.isEmpty() - && rightFilters.isEmpty())) { - return; - } - - // create Filters on top of the children if any filters were - // pushed to them - final RexBuilder rexBuilder = join.getCluster().getRexBuilder(); - final RelBuilder relBuilder = call.builder(); - final RelNode leftRel = - relBuilder.push(join.getLeft()).filter(leftFilters).build(); - final RelNode rightRel = - relBuilder.push(join.getRight()).filter(rightFilters).build(); - - // create the new join node referencing the new children and - // containing its new join filters (if there are any) - final com.google.common.collect.ImmutableList fieldTypes = - com.google.common.collect.ImmutableList.builder() - .addAll(RelOptUtil.getFieldTypeList(leftRel.getRowType())) - .addAll(RelOptUtil.getFieldTypeList(rightRel.getRowType())).build(); - final RexNode joinFilter = - RexUtil.composeConjunction(rexBuilder, - RexUtil.fixUp(rexBuilder, joinFilters, fieldTypes)); - - // If nothing actually got pushed and there is nothing leftover, - // then this rule is a no-op - if (joinFilter.isAlwaysTrue() - && leftFilters.isEmpty() - && rightFilters.isEmpty() - && joinType == join.getJoinType()) { - return; - } - - RelNode newJoinRel = - join.copy( - join.getTraitSet(), - joinFilter, - leftRel, - rightRel, - joinType, - join.isSemiJoinDone()); - call.getPlanner().onCopy(join, newJoinRel); - if (!leftFilters.isEmpty()) { - call.getPlanner().onCopy(filter, leftRel); - } - if (!rightFilters.isEmpty()) { - call.getPlanner().onCopy(filter, rightRel); - } - - relBuilder.push(newJoinRel); - - // Create a project on top of the join if some of the columns have become - // NOT NULL due to the join-type getting stricter. - relBuilder.convert(join.getRowType(), false); - - // create a FilterRel on top of the join if needed - relBuilder.filter( - RexUtil.fixUp(rexBuilder, aboveFilters, - RelOptUtil.getFieldTypeList(relBuilder.peek().getRowType()))); - - call.transformTo(relBuilder.build()); - } - - /** - * Validates that target execution framework can satisfy join filters. - * - *

If the join filter cannot be satisfied (for example, if it is - * {@code l.c1 > r.c2} and the join only supports equi-join), removes the - * filter from {@code joinFilters} and adds it to {@code aboveFilters}. - * - *

The default implementation does nothing; i.e. the join can handle all - * conditions. - * - * @param aboveFilters Filter above Join - * @param joinFilters Filters in join condition - * @param join Join - * @param joinType JoinRelType could be different from type in Join due to - * outer join simplification. - */ - protected void validateJoinFilters(List aboveFilters, - List joinFilters, Join join, JoinRelType joinType) { - final Iterator filterIter = joinFilters.iterator(); - while (filterIter.hasNext()) { - RexNode exp = filterIter.next(); - if (!predicate.apply(join, joinType, exp)) { - aboveFilters.add(exp); - filterIter.remove(); - } - } - } - - /** Rule that pushes parts of the join condition to its inputs. */ - public static class FlinkJoinConditionPushRule extends FlinkFilterJoinRule { - public FlinkJoinConditionPushRule(RelBuilderFactory relBuilderFactory, - Predicate predicate) { - super(RelOptRule.operand(Join.class, RelOptRule.any()), - "FlinkFilterJoinRule:no-filter", true, relBuilderFactory, - predicate); - } - - @Deprecated // to be removed before 2.0 - public FlinkJoinConditionPushRule(RelFactories.FilterFactory filterFactory, - RelFactories.ProjectFactory projectFactory, Predicate predicate) { - this(RelBuilder.proto(filterFactory, projectFactory), predicate); - } - - @Override public void onMatch(RelOptRuleCall call) { - Join join = call.rel(0); - perform(call, null, join); - } - } - - /** Rule that tries to push filter expressions into a join - * condition and into the inputs of the join. */ - public static class FlinkFilterIntoJoinRule extends FlinkFilterJoinRule { - public FlinkFilterIntoJoinRule(boolean smart, - RelBuilderFactory relBuilderFactory, Predicate predicate) { - super( - operand(Filter.class, - operand(Join.class, RelOptRule.any())), - "FlinkFilterJoinRule:filter", smart, relBuilderFactory, - predicate); - } - - @Deprecated // to be removed before 2.0 - public FlinkFilterIntoJoinRule(boolean smart, - RelFactories.FilterFactory filterFactory, - RelFactories.ProjectFactory projectFactory, - Predicate predicate) { - this(smart, RelBuilder.proto(filterFactory, projectFactory), predicate); - } - - @Override public void onMatch(RelOptRuleCall call) { - Filter filter = call.rel(0); - Join join = call.rel(1); - perform(call, filter, join); - } - } - - /** Predicate that returns whether a filter is valid in the ON clause of a - * join for this particular kind of join. If not, Calcite will push it back to - * above the join. */ - public interface Predicate { - boolean apply(Join join, JoinRelType joinType, RexNode exp); - } -} - -// End FlinkFilterJoinRule.java diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinPushExpressionsRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinPushExpressionsRule.java index 310fe8b37d..61b42e71ad 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinPushExpressionsRule.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinPushExpressionsRule.java @@ -17,10 +17,9 @@ package org.apache.flink.table.planner.plan.rules.logical; -import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil; - import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.Join; import org.apache.calcite.rel.core.RelFactories; @@ -66,7 +65,7 @@ public class FlinkJoinPushExpressionsRule extends RelOptRule { Join join = call.rel(0); // Push expression in join condition into Project below Join. - RelNode newJoin = FlinkRelOptUtil.pushDownJoinConditions(join, call.builder()); + RelNode newJoin = RelOptUtil.pushDownJoinConditions(join, call.builder()); // If the join is the same, we bail out if (newJoin instanceof Join) { diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRule.java deleted file mode 100644 index 0d4a9547b0..0000000000 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRule.java +++ /dev/null @@ -1,594 +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.planner.plan.rules.logical; - -import org.apache.calcite.plan.RelOptRule; -import org.apache.calcite.plan.RelOptRuleCall; -import org.apache.calcite.plan.RelOptUtil; -import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.core.Join; -import org.apache.calcite.rel.core.JoinRelType; -import org.apache.calcite.rel.core.RelFactories; -import org.apache.calcite.rel.logical.LogicalJoin; -import org.apache.calcite.rel.rules.MultiJoin; -import org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexInputRef; -import org.apache.calcite.rex.RexNode; -import org.apache.calcite.rex.RexUtil; -import org.apache.calcite.rex.RexVisitorImpl; -import org.apache.calcite.tools.RelBuilderFactory; -import org.apache.calcite.util.ImmutableBitSet; -import org.apache.calcite.util.ImmutableIntList; -import org.apache.calcite.util.Pair; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * This rule is copied from Calcite's {@link org.apache.calcite.rel.rules.JoinToMultiJoinRule}. - * This file should be removed while upgrading Calcite version to 1.21. [CALCITE-3225] - * Modification: - * - Does not match SEMI/ANTI join. lines changed (142-145) - * - lines changed (440-451) - */ - -/** - * Planner rule to flatten a tree of - * {@link org.apache.calcite.rel.logical.LogicalJoin}s - * into a single {@link MultiJoin} with N inputs. - * - *

An input is not flattened if - * the input is a null generating input in an outer join, i.e., either input in - * a full outer join, the right hand side of a left outer join, or the left hand - * side of a right outer join. - * - *

Join conditions are also pulled up from the inputs into the topmost - * {@link MultiJoin}, - * unless the input corresponds to a null generating input in an outer join, - * - *

Outer join information is also stored in the {@link MultiJoin}. A - * boolean flag indicates if the join is a full outer join, and in the case of - * left and right outer joins, the join type and outer join conditions are - * stored in arrays in the {@link MultiJoin}. This outer join information is - * associated with the null generating input in the outer join. So, in the case - * of a a left outer join between A and B, the information is associated with B, - * not A. - * - *

Here are examples of the {@link MultiJoin}s constructed after this rule - * has been applied on following join trees. - * - *

    - *
  • A JOIN B → MJ(A, B) - * - *
  • A JOIN B JOIN C → MJ(A, B, C) - * - *
  • A LEFT JOIN B → MJ(A, B), left outer join on input#1 - * - *
  • A RIGHT JOIN B → MJ(A, B), right outer join on input#0 - * - *
  • A FULL JOIN B → MJ[full](A, B) - * - *
  • A LEFT JOIN (B JOIN C) → MJ(A, MJ(B, C))), left outer join on - * input#1 in the outermost MultiJoin - * - *
  • (A JOIN B) LEFT JOIN C → MJ(A, B, C), left outer join on input#2 - * - *
  • (A LEFT JOIN B) JOIN C → MJ(MJ(A, B), C), left outer join on input#1 - * of the inner MultiJoin TODO - * - *
  • A LEFT JOIN (B FULL JOIN C) → MJ(A, MJ[full](B, C)), left outer join - * on input#1 in the outermost MultiJoin - * - *
  • (A LEFT JOIN B) FULL JOIN (C RIGHT JOIN D) → - * MJ[full](MJ(A, B), MJ(C, D)), left outer join on input #1 in the first - * inner MultiJoin and right outer join on input#0 in the second inner - * MultiJoin - *
- * - *

The constructor is parameterized to allow any sub-class of - * {@link org.apache.calcite.rel.core.Join}, not just - * {@link org.apache.calcite.rel.logical.LogicalJoin}.

- * - * @see org.apache.calcite.rel.rules.FilterMultiJoinMergeRule - * @see org.apache.calcite.rel.rules.ProjectMultiJoinMergeRule - */ -public class FlinkJoinToMultiJoinRule extends RelOptRule { - public static final FlinkJoinToMultiJoinRule INSTANCE = - new FlinkJoinToMultiJoinRule(LogicalJoin.class, RelFactories.LOGICAL_BUILDER); - - //~ Constructors ----------------------------------------------------------- - - @Deprecated // to be removed before 2.0 - public FlinkJoinToMultiJoinRule(Class clazz) { - this(clazz, RelFactories.LOGICAL_BUILDER); - } - - /** - * Creates a FlinkJoinToMultiJoinRule. - */ - public FlinkJoinToMultiJoinRule(Class clazz, - RelBuilderFactory relBuilderFactory) { - super( - operand(clazz, - operand(RelNode.class, any()), - operand(RelNode.class, any())), - relBuilderFactory, null); - } - - //~ Methods ---------------------------------------------------------------- - - @Override - public boolean matches(RelOptRuleCall call) { - final Join origJoin = call.rel(0); - return origJoin.getJoinType() != JoinRelType.SEMI && origJoin.getJoinType() != JoinRelType.ANTI; - } - - public void onMatch(RelOptRuleCall call) { - final Join origJoin = call.rel(0); - final RelNode left = call.rel(1); - final RelNode right = call.rel(2); - - // combine the children MultiJoin inputs into an array of inputs - // for the new MultiJoin - final List projFieldsList = new ArrayList<>(); - final List joinFieldRefCountsList = new ArrayList<>(); - final List newInputs = - combineInputs( - origJoin, - left, - right, - projFieldsList, - joinFieldRefCountsList); - - // combine the outer join information from the left and right - // inputs, and include the outer join information from the current - // join, if it's a left/right outer join - final List> joinSpecs = new ArrayList<>(); - combineOuterJoins( - origJoin, - newInputs, - left, - right, - joinSpecs); - - // pull up the join filters from the children MultiJoinRels and - // combine them with the join filter associated with this LogicalJoin to - // form the join filter for the new MultiJoin - List newJoinFilters = combineJoinFilters(origJoin, left, right); - - // add on the join field reference counts for the join condition - // associated with this LogicalJoin - final com.google.common.collect.ImmutableMap newJoinFieldRefCountsMap = - addOnJoinFieldRefCounts(newInputs, - origJoin.getRowType().getFieldCount(), - origJoin.getCondition(), - joinFieldRefCountsList); - - List newPostJoinFilters = - combinePostJoinFilters(origJoin, left, right); - - final RexBuilder rexBuilder = origJoin.getCluster().getRexBuilder(); - RelNode multiJoin = - new MultiJoin( - origJoin.getCluster(), - newInputs, - RexUtil.composeConjunction(rexBuilder, newJoinFilters), - origJoin.getRowType(), - origJoin.getJoinType() == JoinRelType.FULL, - Pair.right(joinSpecs), - Pair.left(joinSpecs), - projFieldsList, - newJoinFieldRefCountsMap, - RexUtil.composeConjunction(rexBuilder, newPostJoinFilters, true)); - - call.transformTo(multiJoin); - } - - /** - * Combines the inputs into a LogicalJoin into an array of inputs. - * - * @param join original join - * @param left left input into join - * @param right right input into join - * @param projFieldsList returns a list of the new combined projection - * fields - * @param joinFieldRefCountsList returns a list of the new combined join - * field reference counts - * @return combined left and right inputs in an array - */ - private List combineInputs( - Join join, - RelNode left, - RelNode right, - List projFieldsList, - List joinFieldRefCountsList) { - final List newInputs = new ArrayList<>(); - - // leave the null generating sides of an outer join intact; don't - // pull up those children inputs into the array we're constructing - if (canCombine(left, join.getJoinType().generatesNullsOnLeft())) { - final MultiJoin leftMultiJoin = (MultiJoin) left; - for (int i = 0; i < left.getInputs().size(); i++) { - newInputs.add(leftMultiJoin.getInput(i)); - projFieldsList.add(leftMultiJoin.getProjFields().get(i)); - joinFieldRefCountsList.add( - leftMultiJoin.getJoinFieldRefCountsMap().get(i).toIntArray()); - } - } else { - newInputs.add(left); - projFieldsList.add(null); - joinFieldRefCountsList.add( - new int[left.getRowType().getFieldCount()]); - } - - if (canCombine(right, join.getJoinType().generatesNullsOnRight())) { - final MultiJoin rightMultiJoin = (MultiJoin) right; - for (int i = 0; i < right.getInputs().size(); i++) { - newInputs.add(rightMultiJoin.getInput(i)); - projFieldsList.add( - rightMultiJoin.getProjFields().get(i)); - joinFieldRefCountsList.add( - rightMultiJoin.getJoinFieldRefCountsMap().get(i).toIntArray()); - } - } else { - newInputs.add(right); - projFieldsList.add(null); - joinFieldRefCountsList.add( - new int[right.getRowType().getFieldCount()]); - } - - return newInputs; - } - - /** - * Combines the outer join conditions and join types from the left and right - * join inputs. If the join itself is either a left or right outer join, - * then the join condition corresponding to the join is also set in the - * position corresponding to the null-generating input into the join. The - * join type is also set. - * - * @param joinRel join rel - * @param combinedInputs the combined inputs to the join - * @param left left child of the joinrel - * @param right right child of the joinrel - * @param joinSpecs the list where the join types and conditions will be - * copied - */ - private void combineOuterJoins( - Join joinRel, - List combinedInputs, - RelNode left, - RelNode right, - List> joinSpecs) { - JoinRelType joinType = joinRel.getJoinType(); - boolean leftCombined = - canCombine(left, joinType.generatesNullsOnLeft()); - boolean rightCombined = - canCombine(right, joinType.generatesNullsOnRight()); - switch (joinType) { - case LEFT: - if (leftCombined) { - copyOuterJoinInfo( - (MultiJoin) left, - joinSpecs, - 0, - null, - null); - } else { - joinSpecs.add(Pair.of(JoinRelType.INNER, (RexNode) null)); - } - joinSpecs.add(Pair.of(joinType, joinRel.getCondition())); - break; - case RIGHT: - joinSpecs.add(Pair.of(joinType, joinRel.getCondition())); - if (rightCombined) { - copyOuterJoinInfo( - (MultiJoin) right, - joinSpecs, - left.getRowType().getFieldCount(), - right.getRowType().getFieldList(), - joinRel.getRowType().getFieldList()); - } else { - joinSpecs.add(Pair.of(JoinRelType.INNER, (RexNode) null)); - } - break; - default: - if (leftCombined) { - copyOuterJoinInfo( - (MultiJoin) left, - joinSpecs, - 0, - null, - null); - } else { - joinSpecs.add(Pair.of(JoinRelType.INNER, (RexNode) null)); - } - if (rightCombined) { - copyOuterJoinInfo( - (MultiJoin) right, - joinSpecs, - left.getRowType().getFieldCount(), - right.getRowType().getFieldList(), - joinRel.getRowType().getFieldList()); - } else { - joinSpecs.add(Pair.of(JoinRelType.INNER, (RexNode) null)); - } - } - } - - /** - * Copies outer join data from a source MultiJoin to a new set of arrays. - * Also adjusts the conditions to reflect the new position of an input if - * that input ends up being shifted to the right. - * - * @param multiJoin the source MultiJoin - * @param destJoinSpecs the list where the join types and conditions will - * be copied - * @param adjustmentAmount if > 0, the amount the RexInputRefs in the join - * conditions need to be adjusted by - * @param srcFields the source fields that the original join conditions - * are referencing - * @param destFields the destination fields that the new join conditions - */ - private void copyOuterJoinInfo( - MultiJoin multiJoin, - List> destJoinSpecs, - int adjustmentAmount, - List srcFields, - List destFields) { - final List> srcJoinSpecs = - Pair.zip( - multiJoin.getJoinTypes(), - multiJoin.getOuterJoinConditions()); - - if (adjustmentAmount == 0) { - destJoinSpecs.addAll(srcJoinSpecs); - } else { - assert srcFields != null; - assert destFields != null; - int nFields = srcFields.size(); - int[] adjustments = new int[nFields]; - for (int idx = 0; idx < nFields; idx++) { - adjustments[idx] = adjustmentAmount; - } - for (Pair src - : srcJoinSpecs) { - destJoinSpecs.add( - Pair.of( - src.left, - src.right == null - ? null - : src.right.accept( - new RelOptUtil.RexInputConverter( - multiJoin.getCluster().getRexBuilder(), - srcFields, destFields, adjustments)))); - } - } - } - - /** - * Combines the join filters from the left and right inputs (if they are - * MultiJoinRels) with the join filter in the joinrel into a single AND'd - * join filter, unless the inputs correspond to null generating inputs in an - * outer join. - * - * @param joinRel join rel - * @param left left child of the join - * @param right right child of the join - * @return combined join filters AND-ed together - */ - private List combineJoinFilters( - Join joinRel, - RelNode left, - RelNode right) { - JoinRelType joinType = joinRel.getJoinType(); - - // AND the join condition if this isn't a left or right outer join; - // in those cases, the outer join condition is already tracked - // separately - final List filters = new ArrayList<>(); - if ((joinType != JoinRelType.LEFT) && (joinType != JoinRelType.RIGHT)) { - filters.add(joinRel.getCondition()); - } - if (canCombine(left, joinType.generatesNullsOnLeft())) { - filters.add(((MultiJoin) left).getJoinFilter()); - } - // Need to adjust the RexInputs of the right child, since - // those need to shift over to the right - if (canCombine(right, joinType.generatesNullsOnRight())) { - MultiJoin multiJoin = (MultiJoin) right; - filters.add( - shiftRightFilter(joinRel, left, multiJoin, - multiJoin.getJoinFilter())); - } - - return filters; - } - - /** - * Returns whether an input can be merged into a given relational expression - * without changing semantics. - * - * @param input input into a join - * @param nullGenerating true if the input is null generating - * @return true if the input can be combined into a parent MultiJoin - */ - private boolean canCombine(RelNode input, boolean nullGenerating) { - return input instanceof MultiJoin - && !((MultiJoin) input).isFullOuterJoin() - && !(containsOuter((MultiJoin) input)) - && !nullGenerating; - } - - private boolean containsOuter(MultiJoin multiJoin) { - for (JoinRelType joinType : multiJoin.getJoinTypes()) { - if (joinType.isOuterJoin()) { - return true; - } - } - return false; - } - - /** - * Shifts a filter originating from the right child of the LogicalJoin to the - * right, to reflect the filter now being applied on the resulting - * MultiJoin. - * - * @param joinRel the original LogicalJoin - * @param left the left child of the LogicalJoin - * @param right the right child of the LogicalJoin - * @param rightFilter the filter originating from the right child - * @return the adjusted right filter - */ - private RexNode shiftRightFilter( - Join joinRel, - RelNode left, - MultiJoin right, - RexNode rightFilter) { - if (rightFilter == null) { - return null; - } - - int nFieldsOnLeft = left.getRowType().getFieldList().size(); - int nFieldsOnRight = right.getRowType().getFieldList().size(); - int[] adjustments = new int[nFieldsOnRight]; - for (int i = 0; i < nFieldsOnRight; i++) { - adjustments[i] = nFieldsOnLeft; - } - rightFilter = - rightFilter.accept( - new RelOptUtil.RexInputConverter( - joinRel.getCluster().getRexBuilder(), - right.getRowType().getFieldList(), - joinRel.getRowType().getFieldList(), - adjustments)); - return rightFilter; - } - - /** - * Adds on to the existing join condition reference counts the references - * from the new join condition. - * - * @param multiJoinInputs inputs into the new MultiJoin - * @param nTotalFields total number of fields in the MultiJoin - * @param joinCondition the new join condition - * @param origJoinFieldRefCounts existing join condition reference counts - * - * @return Map containing the new join condition - */ - private com.google.common.collect.ImmutableMap addOnJoinFieldRefCounts( - List multiJoinInputs, - int nTotalFields, - RexNode joinCondition, - List origJoinFieldRefCounts) { - // count the input references in the join condition - int[] joinCondRefCounts = new int[nTotalFields]; - joinCondition.accept(new FlinkJoinToMultiJoinRule.InputReferenceCounter(joinCondRefCounts)); - - // first, make a copy of the ref counters - final Map refCountsMap = new HashMap<>(); - int nInputs = multiJoinInputs.size(); - int currInput = 0; - for (int[] origRefCounts : origJoinFieldRefCounts) { - refCountsMap.put( - currInput, - origRefCounts.clone()); - currInput++; - } - - // add on to the counts for each input into the MultiJoin the - // reference counts computed for the current join condition - currInput = -1; - int startField = 0; - int nFields = 0; - for (int i = 0; i < nTotalFields; i++) { - if (joinCondRefCounts[i] == 0) { - continue; - } - while (i >= (startField + nFields)) { - startField += nFields; - currInput++; - assert currInput < nInputs; - nFields = - multiJoinInputs.get(currInput).getRowType().getFieldCount(); - } - int[] refCounts = refCountsMap.get(currInput); - refCounts[i - startField] += joinCondRefCounts[i]; - } - - final com.google.common.collect.ImmutableMap.Builder builder = - com.google.common.collect.ImmutableMap.builder(); - for (Map.Entry entry : refCountsMap.entrySet()) { - builder.put(entry.getKey(), ImmutableIntList.of(entry.getValue())); - } - return builder.build(); - } - - /** - * Combines the post-join filters from the left and right inputs (if they - * are MultiJoinRels) into a single AND'd filter. - * - * @param joinRel the original LogicalJoin - * @param left left child of the LogicalJoin - * @param right right child of the LogicalJoin - * @return combined post-join filters AND'd together - */ - private List combinePostJoinFilters( - Join joinRel, - RelNode left, - RelNode right) { - final List filters = new ArrayList<>(); - if (right instanceof MultiJoin) { - final MultiJoin multiRight = (MultiJoin) right; - filters.add( - shiftRightFilter(joinRel, left, multiRight, - multiRight.getPostJoinFilter())); - } - - if (left instanceof MultiJoin) { - filters.add(((MultiJoin) left).getPostJoinFilter()); - } - - return filters; - } - - //~ Inner Classes ---------------------------------------------------------- - - /** - * Visitor that keeps a reference count of the inputs used by an expression. - */ - private class InputReferenceCounter extends RexVisitorImpl { - private final int[] refCounts; - - InputReferenceCounter(int[] refCounts) { - super(true); - this.refCounts = refCounts; - } - - public Void visitInputRef(RexInputRef inputRef) { - refCounts[inputRef.getIndex()]++; - return null; - } - } -} - -// End FlinkJoinToMultiJoinRule.java diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala index 076242edc3..ab6ca5fb0a 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala @@ -130,9 +130,9 @@ object FlinkBatchRuleSets { */ private val FILTER_RULES: RuleSet = RuleSets.ofList( // push a filter into a join - FlinkFilterJoinRule.FILTER_ON_JOIN, + FilterJoinRule.FILTER_ON_JOIN, // push filter into the children of a join - FlinkFilterJoinRule.JOIN, + FilterJoinRule.JOIN, // push filter through an aggregation FilterAggregateTransposeRule.INSTANCE, // push a filter past a project @@ -223,7 +223,7 @@ object FlinkBatchRuleSets { val JOIN_REORDER_PERPARE_RULES: RuleSet = RuleSets.ofList( // merge join to MultiJoin - FlinkJoinToMultiJoinRule.INSTANCE, + JoinToMultiJoinRule.INSTANCE, // merge project to MultiJoin ProjectMultiJoinMergeRule.INSTANCE, // merge filter to MultiJoin diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala index 1f9b613f4d..3e0210d993 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala @@ -134,9 +134,9 @@ object FlinkStreamRuleSets { */ private val FILTER_RULES: RuleSet = RuleSets.ofList( // push a filter into a join - FlinkFilterJoinRule.FILTER_ON_JOIN, + FilterJoinRule.FILTER_ON_JOIN, // push filter into the children of a join - FlinkFilterJoinRule.JOIN, + FilterJoinRule.JOIN, // push filter through an aggregation FilterAggregateTransposeRule.INSTANCE, // push a filter past a project @@ -210,7 +210,7 @@ object FlinkStreamRuleSets { // merge filter to MultiJoin FilterMultiJoinMergeRule.INSTANCE, // merge join to MultiJoin - FlinkJoinToMultiJoinRule.INSTANCE + JoinToMultiJoinRule.INSTANCE ) val JOIN_REORDER_RULES: RuleSet = RuleSets.ofList( diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala index acf20ecf1d..f3632ea6f1 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala @@ -22,20 +22,15 @@ import org.apache.flink.table.planner.calcite.{FlinkContext, FlinkPlannerImpl, F import org.apache.flink.table.planner.plan.`trait`.{MiniBatchInterval, MiniBatchMode} import org.apache.flink.table.planner.{JBoolean, JByte, JDouble, JFloat, JLong, JShort} -import com.google.common.collect.Lists import org.apache.calcite.config.NullCollation import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.rel.RelFieldCollation.{Direction, NullDirection} -import org.apache.calcite.rel.`type`.RelDataTypeField -import org.apache.calcite.rel.core.{Join, JoinRelType} import org.apache.calcite.rel.{RelFieldCollation, RelNode} import org.apache.calcite.rex.{RexBuilder, RexCall, RexInputRef, RexLiteral, RexNode, RexUtil, RexVisitorImpl} import org.apache.calcite.sql.SqlExplainLevel import org.apache.calcite.sql.SqlKind._ import org.apache.calcite.sql.`type`.SqlTypeName._ -import org.apache.calcite.tools.RelBuilder -import org.apache.calcite.util.mapping.Mappings -import org.apache.calcite.util.{ImmutableBitSet, Pair, Util} +import org.apache.calcite.util.ImmutableBitSet import org.apache.commons.math3.util.ArithmeticUtils import java.io.{PrintWriter, StringWriter} @@ -186,169 +181,6 @@ object FlinkRelOptUtil { } } - /** - * Pushes down expressions in "equal" join condition. - * - * NOTES: This method should be deleted when CALCITE-3171 is fixed. - * - *

For example, given - * "emp JOIN dept ON emp.deptno + 1 = dept.deptno", adds a project above - * "emp" that computes the expression - * "emp.deptno + 1". The resulting join condition is a simple combination - * of AND, equals, and input fields, plus the remaining non-equal conditions. - * - * @param originalJoin Join whose condition is to be pushed down - * @param relBuilder Factory to create project operator - */ - def pushDownJoinConditions(originalJoin: Join, relBuilder: RelBuilder): RelNode = { - var joinCond: RexNode = originalJoin.getCondition - val joinType: JoinRelType = originalJoin.getJoinType - - val extraLeftExprs: util.List[RexNode] = new util.ArrayList[RexNode] - val extraRightExprs: util.List[RexNode] = new util.ArrayList[RexNode] - val leftCount: Int = originalJoin.getLeft.getRowType.getFieldCount - val rightCount: Int = originalJoin.getRight.getRowType.getFieldCount - - // You cannot push a 'get' because field names might change. - // - // Pushing sub-queries is OK in principle (if they don't reference both - // sides of the join via correlating variables) but we'd rather not do it - // yet. - if (!containsGet(joinCond) && RexUtil.SubQueryFinder.find(joinCond) == null) { - joinCond = pushDownEqualJoinConditions( - joinCond, leftCount, rightCount, extraLeftExprs, extraRightExprs) - } - relBuilder.push(originalJoin.getLeft) - if (!extraLeftExprs.isEmpty) { - val fields: util.List[RelDataTypeField] = relBuilder.peek.getRowType.getFieldList - val pairs: util.List[Pair[RexNode, String]] = new util.AbstractList[Pair[RexNode, String]]() { - override def size: Int = leftCount + extraLeftExprs.size - - override def get(index: Int): Pair[RexNode, String] = if (index < leftCount) { - val field: RelDataTypeField = fields.get(index) - Pair.of(new RexInputRef(index, field.getType), field.getName) - } - else Pair.of(extraLeftExprs.get(index - leftCount), null) - } - relBuilder.project(Pair.left(pairs), Pair.right(pairs)) - } - - relBuilder.push(originalJoin.getRight) - if (!extraRightExprs.isEmpty) { - val fields: util.List[RelDataTypeField] = relBuilder.peek.getRowType.getFieldList - val newLeftCount: Int = leftCount + extraLeftExprs.size - val pairs: util.List[Pair[RexNode, String]] = new util.AbstractList[Pair[RexNode, String]]() { - override def size: Int = rightCount + extraRightExprs.size - - override def get(index: Int): Pair[RexNode, String] = if (index < rightCount) { - val field: RelDataTypeField = fields.get(index) - Pair.of(new RexInputRef(index, field.getType), field.getName) - } - else Pair.of(RexUtil.shift(extraRightExprs.get(index - rightCount), -newLeftCount), null) - } - relBuilder.project(Pair.left(pairs), Pair.right(pairs)) - } - - val right: RelNode = relBuilder.build - val left: RelNode = relBuilder.build - relBuilder.push(originalJoin.copy(originalJoin.getTraitSet, joinCond, left, right, joinType, - originalJoin.isSemiJoinDone)) - - // handle SEMI/ANTI join here - var mapping: Mappings.TargetMapping = null - if (!originalJoin.getJoinType.projectsRight()) { - if (!extraLeftExprs.isEmpty) { - mapping = Mappings.createShiftMapping(leftCount + extraLeftExprs.size, 0, 0, leftCount) - } - } else { - if (!extraLeftExprs.isEmpty || !extraRightExprs.isEmpty) { - mapping = Mappings.createShiftMapping( - leftCount + extraLeftExprs.size + rightCount + extraRightExprs.size, - 0, 0, leftCount, leftCount, leftCount + extraLeftExprs.size, rightCount) - } - } - - if (mapping != null) { - relBuilder.project(relBuilder.fields(mapping.inverse)) - } - relBuilder.build - } - - private def containsGet(node: RexNode) = try { - node.accept(new RexVisitorImpl[Void](true) { - override def visitCall(call: RexCall): Void = { - if (call.getOperator eq RexBuilder.GET_OPERATOR) { - throw Util.FoundOne.NULL - } - super.visitCall(call) - } - }) - false - } catch { - case _: Util.FoundOne => - true - } - - /** - * Pushes down parts of a join condition. - * - *

For example, given - * "emp JOIN dept ON emp.deptno + 1 = dept.deptno", adds a project above - * "emp" that computes the expression - * "emp.deptno + 1". The resulting join condition is a simple combination - * of AND, equals, and input fields. - */ - private def pushDownEqualJoinConditions( - node: RexNode, - leftCount: Int, - rightCount: Int, - extraLeftExprs: util.List[RexNode], - extraRightExprs: util.List[RexNode]): RexNode = - node.getKind match { - case AND | EQUALS => - val call = node.asInstanceOf[RexCall] - val list = new util.ArrayList[RexNode] - val operands = Lists.newArrayList(call.getOperands) - // do not use `operands.zipWithIndex.foreach` - operands.indices.foreach { i => - val operand = operands.get(i) - val left2 = leftCount + extraLeftExprs.size - val right2 = rightCount + extraRightExprs.size - val e = pushDownEqualJoinConditions( - operand, leftCount, rightCount, extraLeftExprs, extraRightExprs) - val remainingOperands = Util.skip(operands, i + 1) - val left3 = leftCount + extraLeftExprs.size - fix(remainingOperands, left2, left3) - fix(list, left2, left3) - list.add(e) - } - - if (!(list == call.getOperands)) { - call.clone(call.getType, list) - } else { - call - } - case OR | INPUT_REF | LITERAL | NOT => node - case _ => - if (node.accept(new TimeIndicatorExprFinder)) { - node - } else { - val bits = RelOptUtil.InputFinder.bits(node) - val mid = leftCount + extraLeftExprs.size - Side.of(bits, mid) match { - case Side.LEFT => - fix(extraRightExprs, mid, mid + 1) - extraLeftExprs.add(node) - new RexInputRef(mid, node.getType) - case Side.RIGHT => - val index2 = mid + rightCount + extraRightExprs.size - extraRightExprs.add(node) - new RexInputRef(index2, node.getType) - case _ => node - } - } - } - private def fix(operands: util.List[RexNode], before: Int, after: Int): Unit = { if (before == after) { return diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml index 5d81924213..9af0bbc644 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml @@ -22,10 +22,9 @@ limitations under the License. @@ -47,21 +46,18 @@ Calc(select=[c, g]) @@ -71,10 +67,9 @@ Calc(select=[d, e, f, g, h, a, b, c]) @@ -191,18 +186,17 @@ Calc(select=[a, d]) @@ -312,10 +305,9 @@ HashJoin(joinType=[RightOuterJoin], where=[AND(=(a, d), <(b, h))], select=[d, e, @@ -343,11 +335,11 @@ ON (src1.k = src2.k AND src2.k > 10) ($2, 10))], joinType=[left]) :- LogicalProject(k=[$0], v=[$1]) : +- LogicalFilter(condition=[=($0, 0)]) : +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) - +- LogicalProject(k=[$0], v=[$1], $f2=[>($0, 10)]) + +- LogicalProject(k=[$0], v=[$1]) +- LogicalFilter(condition=[=($0, 0)]) +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml index f6d5a7de6d..cd2c9eb7ea 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml @@ -1843,7 +1843,7 @@ LogicalProject(d=[$0]) (c0, 0), IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(<>(c1, 0), IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) + :- Calc(select=[b, c, CASE(OR(=(c0, 0), AND(IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) : +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], isBroadcast=[true], build=[right]) : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true]) : : :- Calc(select=[a, b, c, c0, ck, i0]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml index f93aa46929..73dbc0f76f 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml @@ -87,19 +87,18 @@ NestedLoopJoin(joinType=[FullOuterJoin], where=[true], select=[d, e, f, g, h, a, @@ -212,10 +208,9 @@ Calc(select=[CASE(IS NOT NULL(a), a, a0) AS $f0, b, c, b0, c0]) @@ -237,21 +232,18 @@ Calc(select=[c, g]) @@ -261,10 +253,9 @@ Calc(select=[d, e, f, g, h, a, b, c]) @@ -467,18 +458,17 @@ NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[d, e, f, g, h, a, @@ -653,10 +642,9 @@ NestedLoopJoin(joinType=[RightOuterJoin], where=[<>(a, d)], select=[d, e, f, g, @@ -682,11 +670,11 @@ ON (src1.k = src2.k AND src2.k > 10) ($2, 10))], joinType=[left]) :- LogicalProject(k=[$0], v=[$1]) : +- LogicalFilter(condition=[=($0, 0)]) : +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) - +- LogicalProject(k=[$0], v=[$1], $f2=[>($0, 10)]) + +- LogicalProject(k=[$0], v=[$1]) +- LogicalFilter(condition=[=($0, 0)]) +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml index 12a5449d07..fa78f5d709 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml @@ -2239,7 +2239,7 @@ LogicalProject(d=[$0]) (c0, 0), IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(<>(c1, 0), IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) + :- Calc(select=[b, c, CASE(OR(=(c0, 0), AND(IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(a, EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], build=[right]) : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true]) : : :- Calc(select=[a, b, c, c0, ck, i0]) @@ -2345,13 +2345,11 @@ LogicalProject(d=[$0]) (b, 10) AS $f3]) - : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) - +- Exchange(distribution=[broadcast]) - +- Calc(select=[d]) - +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(a, d), IS NULL(a), IS NULL(d)), >(b, 10))], select=[a, b, c], build=[right]) +:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- Exchange(distribution=[broadcast]) + +- Calc(select=[d]) + +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) ]]> @@ -2479,7 +2477,7 @@ LogicalProject(e=[$1], d=[$0]) (c0, 0), IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c, 0), AND(<>(c, 0), IS NULL(i), >=(ck0, c), IS NOT NULL(a))), 2, 3) AS $f3]) + :- Calc(select=[b, CASE(OR(=(c0, 0), AND(IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c, 0), AND(IS NULL(i), >=(ck0, c), IS NOT NULL(a))), 2, 3) AS $f3]) : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(a, j)], select=[a, b, c0, ck, i0, c, ck0, j, i], build=[right]) : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c0, ck, i0, c, ck0], build=[right], singleRowJoin=[true]) : : :- Calc(select=[a, b, c AS c0, ck, i0]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml index 09176792b4..a9dfe9d028 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml @@ -2303,7 +2303,7 @@ LogicalProject(d=[$0]) Calc(select=[b]) +- HashJoin(joinType=[LeftAntiJoin], where=[AND(OR(=($f3, d), IS NULL(d)), =(c, f))], select=[b, c, $f3], build=[right]) :- Exchange(distribution=[hash[c]]) - : +- Calc(select=[b, c, CASE(OR(=(c0, 0), AND(<>(c0, 0), IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(<>(c1, 0), IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) + : +- Calc(select=[b, c, CASE(OR(=(c0, 0), AND(IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) : +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], build=[right]) : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true]) : : :- Calc(select=[a, b, c, c0, ck, i0]) @@ -2409,13 +2409,11 @@ LogicalProject(d=[$0]) (b, 10) AS $f3]) - : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) - +- Exchange(distribution=[broadcast]) - +- Calc(select=[d]) - +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(a, d), IS NULL(a), IS NULL(d)), >(b, 10))], select=[a, b, c], build=[right]) +:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- Exchange(distribution=[broadcast]) + +- Calc(select=[d]) + +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) ]]> @@ -2545,7 +2543,7 @@ LogicalProject(e=[$1], d=[$0]) (c0, 0), IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c, 0), AND(<>(c, 0), IS NULL(i), >=(ck0, c), IS NOT NULL(a))), 2, 3) AS $f3]) + :- Calc(select=[b, CASE(OR(=(c0, 0), AND(IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c, 0), AND(IS NULL(i), >=(ck0, c), IS NOT NULL(a))), 2, 3) AS $f3]) : +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, j)], select=[a, b, c0, ck, i0, c, ck0, j, i], build=[right]) : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c0, ck, i0, c, ck0], build=[right], singleRowJoin=[true]) : : :- Calc(select=[a, b, c AS c0, ck, i0]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml index 7afd000635..8c279897d0 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml @@ -22,19 +22,18 @@ limitations under the License. @@ -122,13 +118,13 @@ Calc(select=[CASE(IS NOT NULL(a), a, a0) AS $f0, b, c, b0, c0]) + @@ -151,22 +147,19 @@ Calc(select=[c, g]) @@ -176,10 +169,9 @@ Calc(select=[d, e, f, g, h, a, b, c]) @@ -301,19 +293,18 @@ Calc(select=[a, d]) + @@ -427,10 +418,9 @@ HashJoin(joinType=[RightOuterJoin], where=[AND(=(a, d), <(b, h))], select=[d, e, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml index afcbaaa8cd..5100b1c62b 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml @@ -1897,7 +1897,7 @@ LogicalProject(d=[$0]) Calc(select=[b]) +- HashJoin(joinType=[LeftAntiJoin], where=[AND(OR(=($f3, d), IS NULL(d)), =(c, f))], select=[b, c, $f3], build=[right]) :- Exchange(distribution=[hash[c]]) - : +- Calc(select=[b, c, CASE(OR(=(c0, 0), AND(<>(c0, 0), IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(<>(c1, 0), IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) + : +- Calc(select=[b, c, CASE(OR(=(c0, 0), AND(IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) : +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], build=[right]) : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true]) : : :- Calc(select=[a, b, c, c0, ck, i0]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml index a1c52e2f71..f9ac1e6cfe 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml @@ -22,19 +22,18 @@ limitations under the License. @@ -125,10 +121,9 @@ Calc(select=[CASE(IS NOT NULL(a), a, a0) AS $f0, b, c, b0, c0]) @@ -151,22 +146,19 @@ Calc(select=[c, g]) @@ -176,10 +168,9 @@ Calc(select=[d, e, f, g, h, a, b, c]) @@ -301,19 +292,18 @@ Calc(select=[a, d]) @@ -427,10 +416,9 @@ SortMergeJoin(joinType=[RightOuterJoin], where=[AND(=(a, d), <(b, h))], select=[ @@ -459,11 +447,11 @@ ON (src1.k = src2.k AND src2.k > 10) ($2, 10))], joinType=[left]) :- LogicalProject(k=[$0], v=[$1]) : +- LogicalFilter(condition=[=($0, 0)]) : +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) - +- LogicalProject(k=[$0], v=[$1], $f2=[>($0, 10)]) + +- LogicalProject(k=[$0], v=[$1]) +- LogicalFilter(condition=[=($0, 0)]) +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml index 669d4a6431..54bb7209d9 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml @@ -1982,7 +1982,7 @@ LogicalProject(d=[$0]) Calc(select=[b]) +- SortMergeJoin(joinType=[LeftAntiJoin], where=[AND(OR(=($f3, d), IS NULL(d)), =(c, f))], select=[b, c, $f3]) :- Exchange(distribution=[hash[c]]) - : +- Calc(select=[b, c, CASE(OR(=(c0, 0), AND(<>(c0, 0), IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(<>(c1, 0), IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) + : +- Calc(select=[b, c, CASE(OR(=(c0, 0), AND(IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) : +- SortMergeJoin(joinType=[LeftOuterJoin], where=[=(a, EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i]) : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true]) : : :- Calc(select=[a, b, c, c0, ck, i0]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/JoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/JoinTest.xml index 0bbbc52697..22028912ea 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/JoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/JoinTest.xml @@ -51,9 +51,9 @@ LogicalProject(b=[$1], y=[$4]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.xml index c65896b824..8f253af5ea 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.xml @@ -255,13 +255,11 @@ LogicalProject(c=[$0]) ($1, 2)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) - +- LogicalProject(c=[$0]) - +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) ++- LogicalJoin(condition=[AND(OR(=($0, $2), IS NULL($0), IS NULL($2)), >($1, 2))], joinType=[anti]) + :- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) + +- LogicalProject(c=[$0]) + +- LogicalFilter(condition=[true]) + +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) ]]> @@ -310,13 +308,11 @@ LogicalProject(c=[$0]) ($1, 1)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) - +- LogicalProject(c=[$0], d=[$1]) - +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) ++- LogicalJoin(condition=[AND(OR(=($0, $2), IS NULL($0), IS NULL($2)), =($1, $3), >($1, 1))], joinType=[anti]) + :- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) + +- LogicalProject(c=[$0], d=[$1]) + +- LogicalFilter(condition=[true]) + +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) ]]> @@ -337,13 +333,11 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), >($cor0.b, 2))]) ($1, 2)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) - +- LogicalProject(c=[$0]) - +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) ++- LogicalJoin(condition=[AND(=($0, $2), >($1, 2))], joinType=[anti]) + :- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) + +- LogicalProject(c=[$0]) + +- LogicalFilter(condition=[true]) + +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) ]]> @@ -365,14 +359,12 @@ LogicalProject(c=[$0]) ($1, 2)]) - : +- LogicalFilter(condition=[>($1, 2)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) - +- LogicalProject(c=[$0]) - +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) ++- LogicalJoin(condition=[=($0, $2)], joinType=[semi]) + :- LogicalFilter(condition=[>($1, 2)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) + +- LogicalProject(c=[$0]) + +- LogicalFilter(condition=[true]) + +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) ]]> @@ -421,14 +413,12 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), >($cor0.b, 2))]) ($1, 2)]) - : +- LogicalFilter(condition=[>($1, 2)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) - +- LogicalProject(c=[$0]) - +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) ++- LogicalJoin(condition=[=($0, $2)], joinType=[semi]) + :- LogicalFilter(condition=[>($1, 2)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]]) + +- LogicalProject(c=[$0]) + +- LogicalFilter(condition=[true]) + +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml index b975d4e552..7736deae17 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml @@ -435,13 +435,11 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), >($cor0.b, 10))]) ($1, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - +- LogicalProject(c=[$0]) - +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) ++- LogicalJoin(condition=[AND(=($0, $2), >($1, 10))], joinType=[anti]) + :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalProject(c=[$0]) + +- LogicalFilter(condition=[true]) + +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) ]]> @@ -1006,13 +1004,11 @@ LogicalProject(c=[$0]) ($1, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - +- LogicalProject(c=[$0]) - +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) ++- LogicalJoin(condition=[AND(OR(=($0, $2), IS NULL($0), IS NULL($2)), >($1, 10))], joinType=[anti]) + :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalProject(c=[$0]) + +- LogicalFilter(condition=[true]) + +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) ]]> @@ -1106,7 +1102,7 @@ LogicalProject(d=[$0]) LogicalProject(b=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(OR(=($3, $4), IS NULL($4)), =($2, $5))], joinType=[anti]) - :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(<>($3, 0), IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, 2)]) + :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, 2)]) : +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) @@ -1148,7 +1144,7 @@ LogicalProject(d=[$0]) LogicalProject(b=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(OR(=($3, $4), IS NULL($4)), =($2, $5))], joinType=[anti]) - :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(<>($3, 0), IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, OR(=($7, 0), AND(<>($7, 0), IS NULL($10), >=($8, $7), IS NOT NULL($0))), 2, 3)]) + :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, OR(=($7, 0), AND(IS NULL($10), >=($8, $7), IS NOT NULL($0))), 2, 3)]) : +- LogicalJoin(condition=[=($0, $9)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) @@ -1198,7 +1194,7 @@ LogicalProject(e=[$1], d=[$0]) LogicalProject(b=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(OR(=($1, $4), IS NULL($1), IS NULL($4)), OR(=($3, $5), IS NULL($5)), =($2, $6))], joinType=[anti]) - :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(<>($3, 0), IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, OR(=($7, 0), AND(<>($7, 0), IS NULL($10), >=($8, $7), IS NOT NULL($0))), 2, 3)]) + :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, OR(=($7, 0), AND(IS NULL($10), >=($8, $7), IS NOT NULL($0))), 2, 3)]) : +- LogicalJoin(condition=[=($0, $9)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) @@ -1833,7 +1829,7 @@ LogicalProject(d=[$0]) LogicalProject(b=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[OR(=($3, $4), IS NULL($4))], joinType=[anti]) - :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(<>($3, 0), IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, 2)]) + :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, 2)]) : +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) @@ -1870,7 +1866,7 @@ LogicalProject(d=[$0]) LogicalProject(b=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[OR(=($3, $4), IS NULL($4))], joinType=[anti]) - :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(<>($3, 0), IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, 2)]) + :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, 2)]) : +- LogicalJoin(condition=[=($0, $5)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) @@ -1910,7 +1906,7 @@ LogicalProject(d=[$0]) LogicalProject(b=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[OR(=($3, $4), IS NULL($4))], joinType=[anti]) - :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(<>($3, 0), IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, OR(=($7, 0), AND(<>($7, 0), IS NULL($10), >=($8, $7), IS NOT NULL($0))), 2, 3)]) + :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, OR(=($7, 0), AND(IS NULL($10), >=($8, $7), IS NOT NULL($0))), 2, 3)]) : +- LogicalJoin(condition=[=($0, $9)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) @@ -1958,7 +1954,7 @@ LogicalProject(e=[$1], d=[$0]) LogicalProject(b=[$1]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(OR(=($1, $4), IS NULL($1), IS NULL($4)), OR(=($3, $5), IS NULL($5)))], joinType=[anti]) - :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(<>($3, 0), IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, OR(=($7, 0), AND(<>($7, 0), IS NULL($10), >=($8, $7), IS NOT NULL($0))), 2, 3)]) + :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, OR(=($7, 0), AND(IS NULL($10), >=($8, $7), IS NOT NULL($0))), 2, 3)]) : +- LogicalJoin(condition=[=($0, $9)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) @@ -2006,7 +2002,7 @@ LogicalProject(d=[$0], e=[$1]) LogicalProject(c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[AND(OR(=($3, $5), IS NULL($5)), OR(=($4, $6), IS NULL($6)))], joinType=[anti]) - :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(<>($3, 0), IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, 2)], $f4=[CASE(OR(=($7, 0), AND(<>($7, 0), IS NULL($10), >=($8, $7), IS NOT NULL($1))), 3, 4)]) + :- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[CASE(OR(=($3, 0), AND(IS NULL($6), >=($4, $3), IS NOT NULL($0))), 1, 2)], $f4=[CASE(OR(=($7, 0), AND(IS NULL($10), >=($8, $7), IS NOT NULL($1))), 3, 4)]) : +- LogicalJoin(condition=[=($1, $9)], joinType=[left]) : :- LogicalJoin(condition=[true], joinType=[inner]) : : :- LogicalJoin(condition=[=($0, $5)], joinType=[left]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml index f0531e681d..97b419b754 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml @@ -138,13 +138,11 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), >($cor0.b, 10))]) ($1, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - +- LogicalProject(c=[$0]) - +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) ++- LogicalJoin(condition=[AND(=($0, $2), >($1, 10))], joinType=[semi]) + :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalProject(c=[$0]) + +- LogicalFilter(condition=[true]) + +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) ]]> @@ -1556,13 +1554,11 @@ LogicalProject(c=[$0]) ($1, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) - +- LogicalProject(c=[$0]) - +- LogicalFilter(condition=[true]) - +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) ++- LogicalJoin(condition=[AND(=($0, $2), >($1, 10))], joinType=[semi]) + :- LogicalTableScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b)]]]) + +- LogicalProject(c=[$0]) + +- LogicalFilter(condition=[true]) + +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml index 5d63a64847..49b6a40683 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml @@ -651,19 +651,18 @@ Calc(select=[a1, b1], updateAsRetraction=[false], accMode=[AccRetract]) @@ -1061,11 +1059,11 @@ ON (src1.key = src2.key AND src2.key > 10) ($2, 10))], joinType=[left]) :- LogicalProject(key=[$0], v=[$1]) : +- LogicalFilter(condition=[=($0, 0)]) : +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(key, v)]]]) - +- LogicalProject(key=[$0], v=[$1], $f2=[>($0, 10)]) + +- LogicalProject(key=[$0], v=[$1]) +- LogicalFilter(condition=[=($0, 0)]) +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(key, v)]]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml index 879cfaa4a2..7ea28313c0 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml @@ -2273,7 +2273,7 @@ LogicalProject(d=[$0]) Calc(select=[b]) +- Join(joinType=[LeftAntiJoin], where=[AND(OR(=($f3, d), IS NULL(d)), =(c, f))], select=[b, c, $f3], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[c]]) - : +- Calc(select=[b, c, CASE(OR(=(c0, 0), AND(<>(c0, 0), IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(<>(c1, 0), IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) + : +- Calc(select=[b, c, CASE(OR(=(c0, 0), AND(IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) : +- Join(joinType=[LeftOuterJoin], where=[=(a, EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], leftInputSpec=[NoUniqueKey], rightInputSpec=[HasUniqueKey]) : :- Exchange(distribution=[hash[a]]) : : +- Join(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) @@ -2379,14 +2379,12 @@ LogicalProject(d=[$0]) (b, 10) AS $f3]) - : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) - +- Exchange(distribution=[single]) - +- Calc(select=[d]) - +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +Join(joinType=[LeftAntiJoin], where=[AND(OR(=(a, d), IS NULL(a), IS NULL(d)), >(b, 10))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) +:- Exchange(distribution=[single]) +: +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- Exchange(distribution=[single]) + +- Calc(select=[d]) + +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) ]]> @@ -2519,7 +2517,7 @@ LogicalProject(e=[$1], d=[$0]) Calc(select=[b]) +- Join(joinType=[LeftAntiJoin], where=[AND(OR(=(b, e), IS NULL(b), IS NULL(e)), OR(=($f3, d), IS NULL(d)))], select=[b, $f3], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) - : +- Calc(select=[b, CASE(OR(=(c0, 0), AND(<>(c0, 0), IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(<>(c1, 0), IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) + : +- Calc(select=[b, CASE(OR(=(c0, 0), AND(IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) : +- Join(joinType=[LeftOuterJoin], where=[=(a, j)], select=[a, b, c0, ck, i0, c, ck0, j, i], leftInputSpec=[NoUniqueKey], rightInputSpec=[HasUniqueKey]) : :- Exchange(distribution=[hash[a]]) : : +- Join(joinType=[InnerJoin], where=[true], select=[a, b, c0, ck, i0, c, ck0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/JoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/JoinTest.xml index 437cf16dec..952b2a8988 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/JoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/JoinTest.xml @@ -28,9 +28,9 @@ LogicalProject(b=[$1], y=[$4]) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala index 0c3d30cc30..cbf9077189 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala @@ -56,8 +56,8 @@ class FlinkAggregateJoinTransposeRuleTest extends TableTestBase { .setHepMatchOrder(HepMatchOrder.BOTTOM_UP) .add(RuleSets.ofList( AggregateReduceGroupingRule.INSTANCE, - FlinkFilterJoinRule.FILTER_ON_JOIN, - FlinkFilterJoinRule.JOIN, + FilterJoinRule.FILTER_ON_JOIN, + FilterJoinRule.JOIN, FilterAggregateTransposeRule.INSTANCE, FilterProjectTransposeRule.INSTANCE, FilterMergeRule.INSTANCE, diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.scala index 426740aab9..8b229100d1 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.scala @@ -23,12 +23,12 @@ import org.apache.flink.table.planner.plan.optimize.program.{FlinkBatchProgram, import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder -import org.apache.calcite.rel.rules.FilterProjectTransposeRule +import org.apache.calcite.rel.rules.{FilterJoinRule, FilterProjectTransposeRule} import org.apache.calcite.tools.RuleSets import org.junit.{Before, Test} /** - * Test for [[FlinkFilterJoinRule]]. + * Tests for [[org.apache.calcite.rel.rules.FilterJoinRule]]. */ class FlinkFilterJoinRuleTest extends TableTestBase { private val util = batchTestUtil() @@ -44,8 +44,8 @@ class FlinkFilterJoinRuleTest extends TableTestBase { .setHepMatchOrder(HepMatchOrder.BOTTOM_UP) .add(RuleSets.ofList( FilterProjectTransposeRule.INSTANCE, - FlinkFilterJoinRule.FILTER_ON_JOIN, - FlinkFilterJoinRule.JOIN)) + FilterJoinRule.FILTER_ON_JOIN, + FilterJoinRule.JOIN)) .build() ) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRuleTest.scala index 43647045d0..dca814ce33 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRuleTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRuleTest.scala @@ -23,12 +23,12 @@ import org.apache.flink.table.planner.plan.optimize.program.{FlinkBatchProgram, import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder -import org.apache.calcite.rel.rules.ProjectMultiJoinMergeRule +import org.apache.calcite.rel.rules.{JoinToMultiJoinRule, ProjectMultiJoinMergeRule} import org.apache.calcite.tools.RuleSets import org.junit.{Before, Test} /** - * Test for [[FlinkJoinToMultiJoinRule]]. + * Tests for [[org.apache.calcite.rel.rules.JoinToMultiJoinRule]]. */ class FlinkJoinToMultiJoinRuleTest extends TableTestBase { private val util = batchTestUtil() @@ -43,7 +43,7 @@ class FlinkJoinToMultiJoinRuleTest extends TableTestBase { .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION) .setHepMatchOrder(HepMatchOrder.BOTTOM_UP) .add(RuleSets.ofList( - FlinkJoinToMultiJoinRule.INSTANCE, + JoinToMultiJoinRule.INSTANCE, ProjectMultiJoinMergeRule.INSTANCE)) .build() ) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala index efdd9b2516..0da08f0366 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.optimize.program.{FlinkBatchProgram, import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} import org.apache.calcite.plan.hep.HepMatchOrder +import org.apache.calcite.rel.rules.FilterJoinRule import org.apache.calcite.tools.RuleSets import org.junit.{Before, Test} @@ -45,8 +46,8 @@ class JoinDependentConditionDerivationRuleTest extends TableTestBase { .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE) .setHepMatchOrder(HepMatchOrder.BOTTOM_UP) .add(RuleSets.ofList( - FlinkFilterJoinRule.FILTER_ON_JOIN, - FlinkFilterJoinRule.JOIN, + FilterJoinRule.FILTER_ON_JOIN, + FilterJoinRule.JOIN, JoinDependentConditionDerivationRule.INSTANCE)) .build() ) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala index 5e8e8fef15..3e0e6380b6 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala @@ -23,7 +23,7 @@ import org.apache.flink.table.planner.plan.optimize.program.{BatchOptimizeContex import org.apache.flink.table.planner.utils.TableTestBase import org.apache.calcite.plan.hep.HepMatchOrder -import org.apache.calcite.rel.rules.{FilterMultiJoinMergeRule, JoinToMultiJoinRule, ProjectMultiJoinMergeRule} +import org.apache.calcite.rel.rules.{FilterJoinRule, FilterMultiJoinMergeRule, JoinToMultiJoinRule, ProjectMultiJoinMergeRule} import org.apache.calcite.tools.RuleSets import org.junit.{Before, Test} @@ -43,8 +43,8 @@ class RewriteMultiJoinConditionRuleTest extends TableTestBase { .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION) .setHepMatchOrder(HepMatchOrder.BOTTOM_UP) .add(RuleSets.ofList( - FlinkFilterJoinRule.FILTER_ON_JOIN, - FlinkFilterJoinRule.JOIN)) + FilterJoinRule.FILTER_ON_JOIN, + FilterJoinRule.JOIN)) .build(), "push filter into join") .addProgram(FlinkHepRuleSetProgramBuilder.newBuilder .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlFunction.java new file mode 100644 index 0000000000..5bb2bf5f38 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlFunction.java @@ -0,0 +1,322 @@ +/* + * 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.calcite.sql; + +import org.apache.calcite.linq4j.function.Functions; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.type.SqlOperandTypeChecker; +import org.apache.calcite.sql.type.SqlOperandTypeInference; +import org.apache.calcite.sql.type.SqlReturnTypeInference; +import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.sql.validate.SqlValidatorScope; +import org.apache.calcite.sql.validate.implicit.TypeCoercion; +import org.apache.calcite.util.Util; + +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import javax.annotation.Nonnull; + +import static org.apache.calcite.util.Static.RESOURCE; + +/** + * A SqlFunction is a type of operator which has conventional + * function-call syntax. + * + *

This file is copied from Apache Calcite and should be removed once CALCITE-3360 + * has been fixed. Changed lines: 279-293 + */ +public class SqlFunction extends SqlOperator { + + //~ Instance fields -------------------------------------------------------- + + private final SqlFunctionCategory category; + + private final SqlIdentifier sqlIdentifier; + + private final List paramTypes; + + //~ Constructors ----------------------------------------------------------- + + /** + * Creates a new SqlFunction for a call to a builtin function. + * + * @param name Name of builtin function + * @param kind kind of operator implemented by function + * @param returnTypeInference strategy to use for return type inference + * @param operandTypeInference strategy to use for parameter type inference + * @param operandTypeChecker strategy to use for parameter type checking + * @param category categorization for function + */ + public SqlFunction( + String name, + SqlKind kind, + SqlReturnTypeInference returnTypeInference, + SqlOperandTypeInference operandTypeInference, + SqlOperandTypeChecker operandTypeChecker, + SqlFunctionCategory category) { + // We leave sqlIdentifier as null to indicate + // that this is a builtin. Same for paramTypes. + this(name, null, kind, returnTypeInference, operandTypeInference, + operandTypeChecker, null, category); + + assert !((category == SqlFunctionCategory.USER_DEFINED_CONSTRUCTOR) + && (returnTypeInference == null)); + } + + /** + * Creates a placeholder SqlFunction for an invocation of a function with a + * possibly qualified name. This name must be resolved into either a builtin + * function or a user-defined function. + * + * @param sqlIdentifier possibly qualified identifier for function + * @param returnTypeInference strategy to use for return type inference + * @param operandTypeInference strategy to use for parameter type inference + * @param operandTypeChecker strategy to use for parameter type checking + * @param paramTypes array of parameter types + * @param funcType function category + */ + public SqlFunction( + SqlIdentifier sqlIdentifier, + SqlReturnTypeInference returnTypeInference, + SqlOperandTypeInference operandTypeInference, + SqlOperandTypeChecker operandTypeChecker, + List paramTypes, + SqlFunctionCategory funcType) { + this(Util.last(sqlIdentifier.names), sqlIdentifier, SqlKind.OTHER_FUNCTION, + returnTypeInference, operandTypeInference, operandTypeChecker, + paramTypes, funcType); + } + + /** + * Internal constructor. + */ + protected SqlFunction( + String name, + SqlIdentifier sqlIdentifier, + SqlKind kind, + SqlReturnTypeInference returnTypeInference, + SqlOperandTypeInference operandTypeInference, + SqlOperandTypeChecker operandTypeChecker, + List paramTypes, + SqlFunctionCategory category) { + super(name, kind, 100, 100, returnTypeInference, operandTypeInference, + operandTypeChecker); + + this.sqlIdentifier = sqlIdentifier; + this.category = Objects.requireNonNull(category); + this.paramTypes = + paramTypes == null ? null : ImmutableList.copyOf(paramTypes); + } + + //~ Methods ---------------------------------------------------------------- + + public SqlSyntax getSyntax() { + return SqlSyntax.FUNCTION; + } + + /** + * @return fully qualified name of function, or null for a builtin function + */ + public SqlIdentifier getSqlIdentifier() { + return sqlIdentifier; + } + + @Override public SqlIdentifier getNameAsId() { + if (sqlIdentifier != null) { + return sqlIdentifier; + } + return super.getNameAsId(); + } + + /** + * @return array of parameter types, or null for builtin function + */ + public List getParamTypes() { + return paramTypes; + } + + /** + * Returns a list of parameter names. + * + *

The default implementation returns {@code [arg0, arg1, ..., argN]}. + */ + public List getParamNames() { + return Functions.generate(paramTypes.size(), i -> "arg" + i); + } + + public void unparse( + SqlWriter writer, + SqlCall call, + int leftPrec, + int rightPrec) { + getSyntax().unparse(writer, this, call, leftPrec, rightPrec); + } + + /** + * @return function category + */ + @Nonnull public SqlFunctionCategory getFunctionType() { + return this.category; + } + + /** + * Returns whether this function allows a DISTINCT or + * ALL quantifier. The default is false; some aggregate + * functions return true. + */ + public boolean isQuantifierAllowed() { + return false; + } + + public void validateCall( + SqlCall call, + SqlValidator validator, + SqlValidatorScope scope, + SqlValidatorScope operandScope) { + // This implementation looks for the quantifier keywords DISTINCT or + // ALL as the first operand in the list. If found then the literal is + // not called to validate itself. Further the function is checked to + // make sure that a quantifier is valid for that particular function. + // + // If the first operand does not appear to be a quantifier then the + // parent ValidateCall is invoked to do normal function validation. + + super.validateCall(call, validator, scope, operandScope); + validateQuantifier(validator, call); + } + + /** + * Throws a validation error if a DISTINCT or ALL quantifier is present but + * not allowed. + */ + protected void validateQuantifier(SqlValidator validator, SqlCall call) { + if ((null != call.getFunctionQuantifier()) && !isQuantifierAllowed()) { + throw validator.newValidationError(call.getFunctionQuantifier(), + RESOURCE.functionQuantifierNotAllowed(call.getOperator().getName())); + } + } + + public RelDataType deriveType( + SqlValidator validator, + SqlValidatorScope scope, + SqlCall call) { + return deriveType(validator, scope, call, true); + } + + private RelDataType deriveType( + SqlValidator validator, + SqlValidatorScope scope, + SqlCall call, + boolean convertRowArgToColumnList) { + // Scope for operands. Usually the same as 'scope'. + final SqlValidatorScope operandScope = scope.getOperandScope(call); + + // Indicate to the validator that we're validating a new function call + validator.pushFunctionCall(); + + final List argNames = constructArgNameList(call); + + final List args = constructOperandList(validator, call, argNames); + + final List argTypes = constructArgTypeList(validator, scope, + call, args, convertRowArgToColumnList); + + SqlFunction function = + (SqlFunction) SqlUtil.lookupRoutine(validator.getOperatorTable(), + getNameAsId(), argTypes, argNames, getFunctionType(), + SqlSyntax.FUNCTION, getKind(), + validator.getCatalogReader().nameMatcher(), + false); + try { + // if we have a match on function name and parameter count, but + // couldn't find a function with a COLUMN_LIST type, retry, but + // this time, don't convert the row argument to a COLUMN_LIST type; + // if we did find a match, go back and re-validate the row operands + // (corresponding to column references), now that we can set the + // scope to that of the source cursor referenced by that ColumnList + // type + if (convertRowArgToColumnList && containsRowArg(args)) { + if (function == null + && SqlUtil.matchRoutinesByParameterCount( + validator.getOperatorTable(), getNameAsId(), argTypes, + getFunctionType(), + validator.getCatalogReader().nameMatcher())) { + // remove the already validated node types corresponding to + // row arguments before re-validating + for (SqlNode operand : args) { + if (operand.getKind() == SqlKind.ROW) { + validator.removeValidatedNodeType(operand); + } + } + return deriveType(validator, scope, call, false); + } else if (function != null) { + validator.validateColumnListParams(function, argTypes, args); + } + } + + if (getFunctionType() == SqlFunctionCategory.USER_DEFINED_CONSTRUCTOR) { + return validator.deriveConstructorType(scope, call, this, function, + argTypes); + } + if (function == null) { + // try again if implicit type coercion is allowed. + boolean changed = false; + if (validator.isTypeCoercionEnabled()) { + function = (SqlFunction) SqlUtil.lookupRoutine(validator.getOperatorTable(), + getNameAsId(), argTypes, argNames, getFunctionType(), SqlSyntax.FUNCTION, getKind(), + validator.getCatalogReader().nameMatcher(), + true); + // try to coerce the function arguments to the declared sql type name. + // if we succeed, the arguments would be wrapped with CAST operator. + if (function != null) { + TypeCoercion typeCoercion = validator.getTypeCoercion(); + changed = typeCoercion.userDefinedFunctionCoercion(scope, call, function); + } + } + if (!changed) { + throw validator.handleUnresolvedFunction(call, this, argTypes, + argNames); + } + } + + // REVIEW jvs 25-Mar-2005: This is, in a sense, expanding + // identifiers, but we ignore shouldExpandIdentifiers() + // because otherwise later validation code will + // choke on the unresolved function. + ((SqlBasicCall) call).setOperator(function); + return function.validateOperands( + validator, + operandScope, + call); + } finally { + validator.popFunctionCall(); + } + } + + private boolean containsRowArg(List args) { + for (SqlNode operand : args) { + if (operand.getKind() == SqlKind.ROW) { + return true; + } + } + return false; + } +} + +// End SqlFunction.java diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/JoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/JoinTest.scala index 8574fcfcc8..5709b256d9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/JoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/JoinTest.scala @@ -75,15 +75,15 @@ class JoinTest extends TableTestBase { unaryNode( "DataSetCalc", batchTableNode(table), - term("select", "a", "b", "<(b, 2) AS $f3") + term("select", "a", "b") ), unaryNode( "DataSetCalc", batchTableNode(table1), term("select", "y", "z") ), - term("where", "AND(=(a, z), $f3)"), - term("join", "a", "b", "$f3", "y", "z"), + term("where", "AND(=(a, z), <(b, 2))"), + term("join", "a", "b", "y", "z"), term("joinType", "LeftOuterJoin") ), term("select", "b", "y") @@ -175,10 +175,10 @@ class JoinTest extends TableTestBase { unaryNode( "DataSetCalc", batchTableNode(table1), - term("select", "x", "z", "<(x, 2) AS $f3") + term("select", "x", "z") ), - term("where", "AND(=(a, z), $f3)"), - term("join", "a", "b", "x", "z", "$f3"), + term("where", "AND(=(a, z), <(x, 2))"), + term("join", "a", "b", "x", "z"), term("joinType", "RightOuterJoin") ), term("select", "b", "x") @@ -265,15 +265,15 @@ class JoinTest extends TableTestBase { unaryNode( "DataSetCalc", batchTableNode(table), - term("select", "a", "b", "<(b, 2) AS $f3") + term("select", "a", "b") ), unaryNode( "DataSetCalc", batchTableNode(table1), - term("select", "y", "z", ">(z, 5) AS $f3") + term("select", "y", "z") ), - term("where", "AND(=(a, z), $f3, $f30)"), - term("join", "a", "b", "$f3", "y", "z", "$f30"), + term("where", "AND(=(a, z), <(b, 2), >(z, 5))"), + term("join", "a", "b", "y", "z"), term("joinType", "FullOuterJoin") ), term("select", "b", "y") diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala index 483b42e04b..89ad599233 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala @@ -848,15 +848,15 @@ class JoinTest extends TableTestBase { unaryNode( "DataStreamCalc", streamTableNode(left), - term("select", "a", "b", "<(b, 2) AS $f3") + term("select", "a", "b") ), unaryNode( "DataStreamCalc", streamTableNode(right), term("select", "y", "z") ), - term("where", "AND(=(a, z), $f3)"), - term("join", "a", "b", "$f3", "y", "z"), + term("where", "AND(=(a, z), <(b, 2))"), + term("join", "a", "b", "y", "z"), term("joinType", "LeftOuterJoin") ), term("select", "b", "y") @@ -948,10 +948,10 @@ class JoinTest extends TableTestBase { unaryNode( "DataStreamCalc", streamTableNode(right), - term("select", "x", "z", "<(x, 2) AS $f3") + term("select", "x", "z") ), - term("where", "AND(=(a, z), $f3)"), - term("join", "a", "b", "x", "z", "$f3"), + term("where", "AND(=(a, z), <(x, 2))"), + term("join", "a", "b", "x", "z"), term("joinType", "RightOuterJoin") ), term("select", "b", "x") -- Gitee From 47b0d1569ee8543aaee80b3a5443a112ba5235a0 Mon Sep 17 00:00:00 2001 From: zhengcanbin Date: Sun, 29 Sep 2019 15:57:29 +0800 Subject: [PATCH 081/268] [FLINK-14179][sql-client] Fix the description of 'SHOW FUNCTIONS' in SQL Client (#9752) --- .../main/java/org/apache/flink/table/client/cli/CliStrings.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliStrings.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliStrings.java index 930ebbcb67..1c2fe921a3 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliStrings.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliStrings.java @@ -52,7 +52,7 @@ public final class CliStrings { .append(formatCommand(SqlCommand.RESET, "Resets all session configuration properties.")) .append(formatCommand(SqlCommand.SELECT, "Executes a SQL SELECT query on the Flink cluster.")) .append(formatCommand(SqlCommand.SET, "Sets a session configuration property. Syntax: 'SET =;'. Use 'SET;' for listing all properties.")) - .append(formatCommand(SqlCommand.SHOW_FUNCTIONS, "Shows all registered user-defined functions.")) + .append(formatCommand(SqlCommand.SHOW_FUNCTIONS, "Shows all user-defined and built-in functions.")) .append(formatCommand(SqlCommand.SHOW_TABLES, "Shows all registered tables.")) .append(formatCommand(SqlCommand.SOURCE, "Reads a SQL SELECT query from a file and executes it on the Flink cluster.")) .append(formatCommand(SqlCommand.USE_CATALOG, "Sets the current catalog. The current database is set to the catalog's default one. Experimental! Syntax: 'USE CATALOG ;'")) -- Gitee From 6a67f367188071e0e076caae24948f52018725d3 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Wed, 25 Sep 2019 17:30:44 +0800 Subject: [PATCH 082/268] [hotfix][python] Use pyflink code with higher priority if it exists Add directory flink-python/pyflink to PYTHONPATH if it exists, this is helpful during development as this script is used to start up the Python worker and putting the directory of flink-python/pyflink to PYTHONPATH makes sure the Python source code will take effect immediately after changed. --- flink-python/bin/pyflink-udf-runner.sh | 22 +++++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/flink-python/bin/pyflink-udf-runner.sh b/flink-python/bin/pyflink-udf-runner.sh index d2f96e875a..38c38cd314 100755 --- a/flink-python/bin/pyflink-udf-runner.sh +++ b/flink-python/bin/pyflink-udf-runner.sh @@ -32,11 +32,27 @@ if [[ "$python" = "" ]]; then python="python" fi -# Add pyflink & py4j to PYTHONPATH -PYFLINK_ZIP="$FLINK_OPT_DIR/python/pyflink.zip" -if [[ ! ${PYTHONPATH} =~ ${PYFLINK_ZIP} ]]; then +CURRENT_DIR=`pwd -P` +FLINK_SOURCE_ROOT_DIR=`cd $bin/../../../../../; pwd -P` +cd $CURRENT_DIR + +# Add pyflink to PYTHONPATH +FLINK_PYTHON="${FLINK_SOURCE_ROOT_DIR}/flink-python" +if [[ ! -f "${FLINK_PYTHON}/pyflink/fn_execution/boot.py" ]]; then + # Add pyflink.zip to PYTHONPATH if directory pyflink does not exist + PYFLINK_ZIP="$FLINK_OPT_DIR/python/pyflink.zip" + if [[ ! ${PYTHONPATH} =~ ${PYFLINK_ZIP} ]]; then export PYTHONPATH="$PYFLINK_ZIP:$PYTHONPATH" + fi +else + # Add directory flink-python/pyflink to PYTHONPATH if it exists, this is helpful during + # development as this script is used to start up the Python worker and putting the + # directory of flink-python/pyflink to PYTHONPATH makes sure the Python source code will + # take effect immediately after changed. + export PYTHONPATH="$FLINK_PYTHON:$PYTHONPATH" fi + +# Add py4j to PYTHONPATH PY4J_ZIP=`echo "$FLINK_OPT_DIR"/python/py4j-*-src.zip` if [[ ! ${PYTHONPATH} =~ ${PY4J_ZIP} ]]; then export PYTHONPATH="$PY4J_ZIP:$PYTHONPATH" -- Gitee From 06d6dba99504f142f08615951d69f90b2d6d3d69 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 26 Sep 2019 09:48:04 +0800 Subject: [PATCH 083/268] [FLINK-14018][python] Add Python building blocks to make sure the basic functionality of Python ScalarFunction could work With this commit, users can write end to end python Table Api jobs with python UDFs. This commit mainly includes the Api, rule optimization and python runtime changes. --- .gitignore | 2 + flink-python/pom.xml | 20 ++ flink-python/pyflink/fn_execution/boot.py | 2 +- .../pyflink/fn_execution/coder_impl.py | 80 ++++++ flink-python/pyflink/fn_execution/coders.py | 86 ++++++ .../pyflink/fn_execution/operations.py | 261 ++++++++++++++++++ .../pyflink/fn_execution/sdk_worker_main.py | 30 ++ flink-python/pyflink/gen_protos.py | 146 ++++++++++ flink-python/pyflink/table/__init__.py | 10 +- .../pyflink/table/table_environment.py | 30 ++ .../tests/test_environment_completeness.py | 3 +- flink-python/pyflink/table/tests/test_udf.py | 245 ++++++++++++++++ flink-python/pyflink/table/types.py | 2 +- flink-python/pyflink/table/udf.py | 229 +++++++++++++++ .../pyflink/testing/test_case_utils.py | 5 +- flink-python/setup.py | 37 ++- .../python/AbstractPythonFunctionRunner.java | 12 +- .../AbstractPythonScalarFunctionOperator.java | 10 +- .../BaseRowPythonScalarFunctionOperator.java | 19 +- .../python/PythonScalarFunctionOperator.java | 20 +- ...bstractPythonScalarFunctionRunnerTest.java | 2 +- ...BaseRowPythonScalarFunctionRunnerTest.java | 2 +- .../PythonScalarFunctionRunnerTest.java | 4 +- ...seRowPythonScalarFunctionOperatorTest.java | 8 +- .../PythonScalarFunctionOperatorTest.java | 8 +- .../PythonScalarFunctionOperatorTestBase.java | 4 +- flink-python/tox.ini | 2 +- .../table/functions/python/PythonEnv.java | 25 +- .../python/SimplePythonFunction.java | 56 ++++ .../codegen/PythonFunctionCodeGenerator.scala | 128 +++++++++ .../table/plan/nodes/CommonPythonCalc.scala | 75 +++++ .../datastream/DataStreamPythonCalc.scala | 125 ++++++++- .../datastream/DataStreamPythonCalcRule.scala | 2 +- .../table/util/python/PythonTableUtils.scala | 29 ++ pom.xml | 1 + 35 files changed, 1633 insertions(+), 87 deletions(-) create mode 100644 flink-python/pyflink/fn_execution/coder_impl.py create mode 100644 flink-python/pyflink/fn_execution/coders.py create mode 100644 flink-python/pyflink/fn_execution/operations.py create mode 100644 flink-python/pyflink/fn_execution/sdk_worker_main.py create mode 100644 flink-python/pyflink/gen_protos.py create mode 100644 flink-python/pyflink/table/tests/test_udf.py create mode 100644 flink-python/pyflink/table/udf.py create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/SimplePythonFunction.java create mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala create mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCalc.scala diff --git a/.gitignore b/.gitignore index 88c4488140..4e3a5ae2f2 100644 --- a/.gitignore +++ b/.gitignore @@ -35,6 +35,8 @@ flink-python/dev/download flink-python/dev/.conda/ flink-python/dev/log/ flink-python/dev/.stage.txt +flink-python/.eggs/ +flink-python/pyflink/fn_execution/*_pb2.py atlassian-ide-plugin.xml out/ /docs/api diff --git a/flink-python/pom.xml b/flink-python/pom.xml index ed16daa09e..ffeb4864c5 100644 --- a/flink-python/pom.xml +++ b/flink-python/pom.xml @@ -327,6 +327,26 @@ under the License. + + exec-maven-plugin + org.codehaus.mojo + 1.5.0 + + + Protos Generation + generate-sources + + exec + + + python + + ${basedir}/pyflink/gen_protos.py + + + + + diff --git a/flink-python/pyflink/fn_execution/boot.py b/flink-python/pyflink/fn_execution/boot.py index e42ad2eea5..bf913846fb 100644 --- a/flink-python/pyflink/fn_execution/boot.py +++ b/flink-python/pyflink/fn_execution/boot.py @@ -145,5 +145,5 @@ env = dict(os.environ) if "FLINK_BOOT_TESTING" in os.environ and os.environ["FLINK_BOOT_TESTING"] == "1": exit(0) -call([sys.executable, "-m", "apache_beam.runners.worker.sdk_worker_main"], +call([sys.executable, "-m", "pyflink.fn_execution.sdk_worker_main"], stdout=sys.stdout, stderr=sys.stderr, env=env) diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py new file mode 100644 index 0000000000..2f7c5d3ab6 --- /dev/null +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -0,0 +1,80 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +import sys + +from apache_beam.coders.coder_impl import StreamCoderImpl + +if sys.version > '3': + xrange = range + + +class RowCoderImpl(StreamCoderImpl): + + def __init__(self, field_coders): + self._field_coders = field_coders + + def encode_to_stream(self, value, out_stream, nested): + self.write_null_mask(value, out_stream) + for i in xrange(len(self._field_coders)): + self._field_coders[i].encode_to_stream(value[i], out_stream, nested) + + def decode_from_stream(self, in_stream, nested): + from pyflink.table import Row + null_mask = self.read_null_mask(len(self._field_coders), in_stream) + assert len(null_mask) == len(self._field_coders) + return Row(*[None if null_mask[idx] else self._field_coders[idx].decode_from_stream( + in_stream, nested) for idx in xrange(0, len(null_mask))]) + + @staticmethod + def write_null_mask(value, out_stream): + field_pos = 0 + field_count = len(value) + while field_pos < field_count: + b = 0x00 + # set bits in byte + num_pos = min(8, field_count - field_pos) + byte_pos = 0 + while byte_pos < num_pos: + b = b << 1 + # set bit if field is null + if value[field_pos + byte_pos] is None: + b |= 0x01 + byte_pos += 1 + field_pos += num_pos + # shift bits if last byte is not completely filled + b <<= (8 - byte_pos) + # write byte + out_stream.write_byte(b) + + @staticmethod + def read_null_mask(field_count, in_stream): + null_mask = [] + field_pos = 0 + while field_pos < field_count: + b = in_stream.read_byte() + num_pos = min(8, field_count - field_pos) + byte_pos = 0 + while byte_pos < num_pos: + null_mask.append((b & 0x80) > 0) + b = b << 1 + byte_pos += 1 + field_pos += num_pos + return null_mask + + def __repr__(self): + return 'RowCoderImpl[%s]' % ', '.join(str(c) for c in self._field_coders) diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py new file mode 100644 index 0000000000..c6add86477 --- /dev/null +++ b/flink-python/pyflink/fn_execution/coders.py @@ -0,0 +1,86 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +import sys + +from apache_beam.coders import Coder, VarIntCoder +from apache_beam.coders.coders import FastCoder + +from pyflink.fn_execution import coder_impl +from pyflink.fn_execution import flink_fn_execution_pb2 + +FLINK_SCHEMA_CODER_URN = "flink:coder:schema:v1" + +if sys.version > '3': + xrange = range + + +__all__ = ['RowCoder'] + + +class RowCoder(FastCoder): + """ + Coder for Row. + """ + + def __init__(self, field_coders): + self._field_coders = field_coders + + def _create_impl(self): + return coder_impl.RowCoderImpl([c.get_impl() for c in self._field_coders]) + + def is_deterministic(self): + return all(c.is_deterministic() for c in self._field_coders) + + def to_type_hint(self): + from pyflink.table import Row + return Row + + def __repr__(self): + return 'RowCoder[%s]' % ', '.join(str(c) for c in self._field_coders) + + def __eq__(self, other): + return (self.__class__ == other.__class__ + and len(self._field_coders) == len(other._field_coders) + and [self._field_coders[i] == other._field_coders[i] for i in + xrange(len(self._field_coders))]) + + def __ne__(self, other): + return not self == other + + def __hash__(self): + return hash(self._field_coders) + + +@Coder.register_urn(FLINK_SCHEMA_CODER_URN, flink_fn_execution_pb2.Schema) +def _pickle_from_runner_api_parameter(schema_proto, unused_components, unused_context): + return RowCoder([from_proto(f.type) for f in schema_proto.fields]) + + +def from_proto(field_type): + """ + Creates the corresponding :class:`Coder` given the protocol representation of the field type. + + :param field_type: the protocol representation of the field type + :return: :class:`Coder` + """ + if field_type.type_name == flink_fn_execution_pb2.Schema.TypeName.BIGINT: + return VarIntCoder() + elif field_type.type_name == flink_fn_execution_pb2.Schema.TypeName.ROW: + return RowCoder([from_proto(f.type) for f in field_type.row_schema.fields]) + else: + raise ValueError("field_type %s is not supported." % field_type) diff --git a/flink-python/pyflink/fn_execution/operations.py b/flink-python/pyflink/fn_execution/operations.py new file mode 100644 index 0000000000..13e179bb53 --- /dev/null +++ b/flink-python/pyflink/fn_execution/operations.py @@ -0,0 +1,261 @@ +################################################################################ +# 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. +################################################################################ + +from abc import abstractmethod, ABCMeta + +from apache_beam.runners.worker import operation_specs +from apache_beam.runners.worker import bundle_processor +from apache_beam.runners.worker.operations import Operation + +from pyflink.fn_execution import flink_fn_execution_pb2 + +SCALAR_FUNCTION_URN = "flink:transform:scalar_function:v1" + + +class InputGetter(object): + """ + Base class for get an input argument for a :class:`UserDefinedFunction`. + """ + __metaclass__ = ABCMeta + + def open(self): + pass + + def close(self): + pass + + @abstractmethod + def get(self, value): + pass + + +class OffsetInputGetter(InputGetter): + """ + InputGetter for the input argument which is a column of the input row. + + :param input_offset: the offset of the column in the input row + """ + + def __init__(self, input_offset): + self.input_offset = input_offset + + def get(self, value): + return value[self.input_offset] + + +class ScalarFunctionInputGetter(InputGetter): + """ + InputGetter for the input argument which is a Python :class:`ScalarFunction`. This is used for + chaining Python functions. + + :param scalar_function_proto: the proto representation of the Python :class:`ScalarFunction` + """ + + def __init__(self, scalar_function_proto): + self.scalar_function_invoker = create_scalar_function_invoker(scalar_function_proto) + + def open(self): + self.scalar_function_invoker.invoke_open() + + def close(self): + self.scalar_function_invoker.invoke_close() + + def get(self, value): + return self.scalar_function_invoker.invoke_eval(value) + + +class ScalarFunctionInvoker(object): + """ + An abstraction that can be used to execute :class:`ScalarFunction` methods. + + A ScalarFunctionInvoker describes a particular way for invoking methods of a + :class:`ScalarFunction`. + + :param scalar_function: the :class:`ScalarFunction` to execute + :param inputs: the input arguments for the :class:`ScalarFunction` + """ + + def __init__(self, scalar_function, inputs): + self.scalar_function = scalar_function + self.input_getters = [] + for input in inputs: + if input.HasField("udf"): + # for chaining Python UDF input: the input argument is a Python ScalarFunction + self.input_getters.append(ScalarFunctionInputGetter(input.udf)) + else: + # the input argument is a column of the input row + self.input_getters.append(OffsetInputGetter(input.inputOffset)) + + def invoke_open(self): + """ + Invokes the ScalarFunction.open() function. + """ + for input_getter in self.input_getters: + input_getter.open() + # set the FunctionContext to None for now + self.scalar_function.open(None) + + def invoke_close(self): + """ + Invokes the ScalarFunction.close() function. + """ + for input_getter in self.input_getters: + input_getter.close() + self.scalar_function.close() + + def invoke_eval(self, value): + """ + Invokes the ScalarFunction.eval() function. + + :param value: the input element for which eval() method should be invoked + """ + args = [input_getter.get(value) for input_getter in self.input_getters] + return self.scalar_function.eval(*args) + + +def create_scalar_function_invoker(scalar_function_proto): + """ + Creates :class:`ScalarFunctionInvoker` from the proto representation of a + :class:`ScalarFunction`. + + :param scalar_function_proto: the proto representation of the Python :class:`ScalarFunction` + :return: :class:`ScalarFunctionInvoker`. + """ + import cloudpickle + scalar_function = cloudpickle.loads(scalar_function_proto.payload) + return ScalarFunctionInvoker(scalar_function, scalar_function_proto.inputs) + + +class ScalarFunctionRunner(object): + """ + The runner which is responsible for executing the scalar functions and send the + execution results back to the remote Java operator. + + :param udfs_proto: protocol representation for the scalar functions to execute + """ + + def __init__(self, udfs_proto): + self.scalar_function_invokers = [create_scalar_function_invoker(f) for f in + udfs_proto] + + def setup(self, main_receivers): + """ + Set up the ScalarFunctionRunner. + + :param main_receivers: Receiver objects which is responsible for sending the execution + results back the the remote Java operator + """ + from apache_beam.runners.common import _OutputProcessor + self.output_processor = _OutputProcessor( + window_fn=None, + main_receivers=main_receivers, + tagged_receivers=None, + per_element_output_counter=None) + + def open(self): + for invoker in self.scalar_function_invokers: + invoker.invoke_open() + + def close(self): + for invoker in self.scalar_function_invokers: + invoker.invoke_close() + + def process(self, windowed_value): + results = [invoker.invoke_eval(windowed_value.value) for invoker in + self.scalar_function_invokers] + from pyflink.table import Row + result = Row(*results) + # send the execution results back + self.output_processor.process_outputs(windowed_value, [result]) + + +class ScalarFunctionOperation(Operation): + """ + An operation that will execute ScalarFunctions for each input element. + """ + + def __init__(self, name, spec, counter_factory, sampler, consumers): + super(ScalarFunctionOperation, self).__init__(name, spec, counter_factory, sampler) + for tag, op_consumers in consumers.items(): + for consumer in op_consumers: + self.add_receiver(consumer, 0) + + self.scalar_function_runner = ScalarFunctionRunner(self.spec.serialized_fn) + self.scalar_function_runner.open() + + def setup(self): + with self.scoped_start_state: + super(ScalarFunctionOperation, self).setup() + self.scalar_function_runner.setup(self.receivers[0]) + + def start(self): + with self.scoped_start_state: + super(ScalarFunctionOperation, self).start() + + def process(self, o): + with self.scoped_process_state: + self.scalar_function_runner.process(o) + + def finish(self): + with self.scoped_finish_state: + super(ScalarFunctionOperation, self).finish() + + def needs_finalization(self): + return False + + def reset(self): + super(ScalarFunctionOperation, self).reset() + + def teardown(self): + with self.scoped_finish_state: + self.scalar_function_runner.close() + + def progress_metrics(self): + metrics = super(ScalarFunctionOperation, self).progress_metrics() + metrics.processed_elements.measured.output_element_counts.clear() + tag = None + receiver = self.receivers[0] + metrics.processed_elements.measured.output_element_counts[ + str(tag)] = receiver.opcounter.element_counter.value() + return metrics + + +@bundle_processor.BeamTransformFactory.register_urn( + SCALAR_FUNCTION_URN, flink_fn_execution_pb2.UserDefinedFunctions) +def create(factory, transform_id, transform_proto, parameter, consumers): + return _create_user_defined_function_operation( + factory, transform_proto, consumers, parameter.udfs) + + +def _create_user_defined_function_operation(factory, transform_proto, consumers, udfs_proto, + operation_cls=ScalarFunctionOperation): + output_tags = list(transform_proto.outputs.keys()) + output_coders = factory.get_output_coders(transform_proto) + spec = operation_specs.WorkerDoFn( + serialized_fn=udfs_proto, + output_tags=output_tags, + input=None, + side_inputs=None, + output_coders=[output_coders[tag] for tag in output_tags]) + + return operation_cls( + transform_proto.unique_name, + spec, + factory.counter_factory, + factory.state_sampler, + consumers) diff --git a/flink-python/pyflink/fn_execution/sdk_worker_main.py b/flink-python/pyflink/fn_execution/sdk_worker_main.py new file mode 100644 index 0000000000..82d091c529 --- /dev/null +++ b/flink-python/pyflink/fn_execution/sdk_worker_main.py @@ -0,0 +1,30 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import sys + +# force to register the operations to SDK Harness +import pyflink.fn_execution.operations # noqa # pylint: disable=unused-import + +# force to register the coders to SDK Harness +import pyflink.fn_execution.coders # noqa # pylint: disable=unused-import + +import apache_beam.runners.worker.sdk_worker_main + +if __name__ == '__main__': + apache_beam.runners.worker.sdk_worker_main.main(sys.argv) diff --git a/flink-python/pyflink/gen_protos.py b/flink-python/pyflink/gen_protos.py new file mode 100644 index 0000000000..ce4ddd4010 --- /dev/null +++ b/flink-python/pyflink/gen_protos.py @@ -0,0 +1,146 @@ +################################################################################ +# 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. +################################################################################ + +from __future__ import absolute_import +from __future__ import print_function + +import glob +import logging +import multiprocessing +import os +import platform +import shutil +import subprocess +import sys +import time +import warnings + +import pkg_resources + +# latest grpcio-tools incompatible with latest protobuf 3.6.1. +GRPC_TOOLS = 'grpcio-tools>=1.3.5,<=1.14.2' + +PROTO_PATHS = [ + os.path.join('proto'), +] + +PYTHON_OUTPUT_PATH = os.path.join('fn_execution') + + +def generate_proto_files(force=False): + try: + import grpc_tools # noqa # pylint: disable=unused-import + except ImportError: + warnings.warn('Installing grpcio-tools is recommended for development.') + + py_sdk_root = os.path.dirname(os.path.abspath(__file__)) + proto_dirs = [os.path.join(py_sdk_root, path) for path in PROTO_PATHS] + proto_files = sum( + [glob.glob(os.path.join(d, '*.proto')) for d in proto_dirs], []) + out_dir = os.path.join(py_sdk_root, PYTHON_OUTPUT_PATH) + out_files = [path for path in glob.glob(os.path.join(out_dir, '*_pb2.py'))] + + if out_files and not proto_files and not force: + # We have out_files but no protos; assume they're up to date. + # This is actually the common case (e.g. installation from an sdist). + logging.info('No proto files; using existing generated files.') + return + + elif not out_files and not proto_files: + raise RuntimeError( + 'No proto files found in %s.' % proto_dirs) + + # Regenerate iff the proto files or this file are newer. + elif force or not out_files or len(out_files) < len(proto_files) or ( + min(os.path.getmtime(path) for path in out_files) + <= max(os.path.getmtime(path) + for path in proto_files + [os.path.realpath(__file__)])): + try: + from grpc_tools import protoc + except ImportError: + if platform.system() == 'Windows': + # For Windows, grpcio-tools has to be installed manually. + raise RuntimeError( + 'Cannot generate protos for Windows since grpcio-tools package is ' + 'not installed. Please install this package manually ' + 'using \'pip install grpcio-tools\'.') + + # Use a subprocess to avoid messing with this process' path and imports. + # Note that this requires a separate module from setup.py for Windows: + # https://docs.python.org/2/library/multiprocessing.html#windows + p = multiprocessing.Process( + target=_install_grpcio_tools_and_generate_proto_files) + p.start() + p.join() + if p.exitcode: + raise ValueError("Proto generation failed (see log for details).") + else: + logging.info('Regenerating out-of-date Python proto definitions.') + builtin_protos = pkg_resources.resource_filename('grpc_tools', '_proto') + args = ( + [sys.executable] + # expecting to be called from command line + ['--proto_path=%s' % builtin_protos] + + ['--proto_path=%s' % d for d in proto_dirs] + + ['--python_out=%s' % out_dir] + + proto_files) + ret_code = protoc.main(args) + if ret_code: + raise RuntimeError( + 'Protoc returned non-zero status (see logs for details): ' + '%s' % ret_code) + + +# Though wheels are available for grpcio-tools, setup_requires uses +# easy_install which doesn't understand them. This means that it is +# compiled from scratch (which is expensive as it compiles the full +# protoc compiler). Instead, we attempt to install a wheel in a temporary +# directory and add it to the path as needed. +# See https://github.com/pypa/setuptools/issues/377 +def _install_grpcio_tools_and_generate_proto_files(): + install_path = os.path.join( + os.path.dirname(os.path.abspath(__file__)), '..', '.eggs', 'grpcio-wheels') + build_path = install_path + '-build' + if os.path.exists(build_path): + shutil.rmtree(build_path) + logging.warning('Installing grpcio-tools into %s', install_path) + try: + start = time.time() + subprocess.check_call( + [sys.executable, '-m', 'pip', 'install', + '--prefix', install_path, '--build', build_path, + '--upgrade', GRPC_TOOLS, "-I"]) + from distutils.dist import Distribution + install_obj = Distribution().get_command_obj('install', create=True) + install_obj.prefix = install_path + install_obj.finalize_options() + logging.warning( + 'Installing grpcio-tools took %0.2f seconds.', time.time() - start) + finally: + sys.stderr.flush() + shutil.rmtree(build_path, ignore_errors=True) + sys.path.append(install_obj.install_purelib) + if install_obj.install_purelib != install_obj.install_platlib: + sys.path.append(install_obj.install_platlib) + try: + generate_proto_files() + finally: + sys.stderr.flush() + + +if __name__ == '__main__': + generate_proto_files(force=True) diff --git a/flink-python/pyflink/table/__init__.py b/flink-python/pyflink/table/__init__.py index e69a9b7ecb..82ce28a340 100644 --- a/flink-python/pyflink/table/__init__.py +++ b/flink-python/pyflink/table/__init__.py @@ -52,6 +52,11 @@ Important classes of Flink Table API: from a registered :class:`pyflink.table.catalog.Catalog`. - :class:`pyflink.table.TableSchema` Represents a table's structure with field names and data types. + - :class:`pyflink.table.FunctionContext` + Used to obtain global runtime information about the context in which the + user-defined function is executed, such as the metric group, and global job parameters, etc. + - :class:`pyflink.table.ScalarFunction` + Base interface for user-defined scalar function. """ from __future__ import absolute_import @@ -65,6 +70,7 @@ from pyflink.table.sinks import TableSink, CsvTableSink, WriteMode from pyflink.table.sources import TableSource, CsvTableSource from pyflink.table.types import DataTypes, UserDefinedType, Row from pyflink.table.table_schema import TableSchema +from pyflink.table.udf import FunctionContext, ScalarFunction __all__ = [ 'TableEnvironment', @@ -85,5 +91,7 @@ __all__ = [ 'DataTypes', 'UserDefinedType', 'Row', - 'TableSchema' + 'TableSchema', + 'FunctionContext', + 'ScalarFunction' ] diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index 33160e0cdb..940aafcf62 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -542,6 +542,36 @@ class TableEnvironment(object): .loadClass(function_class_name).newInstance() self._j_tenv.registerFunction(name, java_function) + def register_function(self, name, function): + """ + Registers a python user-defined function under a unique name. Replaces already existing + user-defined function under this name. + + Example: + :: + + >>> table_env.register_function( + ... "add_one", udf(lambda i: i + 1, DataTypes.BIGINT(), DataTypes.BIGINT())) + + >>> @udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], + ... result_type=DataTypes.BIGINT()) + ... def add(i, j): + ... return i + j + >>> table_env.register_function("add", add) + + >>> class SubtractOne(ScalarFunction): + ... def eval(self, i): + ... return i - 1 + >>> table_env.register_function( + ... "subtract_one", udf(SubtractOne(), DataTypes.BIGINT(), DataTypes.BIGINT())) + + :param name: The name under which the function is registered. + :type name: str + :param function: The python user-defined function to register. + :type function: UserDefinedFunctionWrapper + """ + self._j_tenv.registerFunction(name, function._judf) + def execute(self, job_name): """ Triggers the program execution. The environment will execute all parts of diff --git a/flink-python/pyflink/table/tests/test_environment_completeness.py b/flink-python/pyflink/table/tests/test_environment_completeness.py index 5f73c3c88b..89459891f8 100644 --- a/flink-python/pyflink/table/tests/test_environment_completeness.py +++ b/flink-python/pyflink/table/tests/test_environment_completeness.py @@ -41,8 +41,7 @@ class EnvironmentAPICompletenessTests(PythonAPICompletenessTestCase, unittest.Te # registerCatalog, getCatalog and listTables should be supported when catalog supported in # python. getCompletionHints has been deprecated. It will be removed in the next release. # TODO add TableEnvironment#create method with EnvironmentSettings as a parameter - return {'registerCatalog', 'getCatalog', 'registerFunction', 'listTables', - 'getCompletionHints', 'create'} + return {'registerCatalog', 'getCatalog', 'listTables', 'getCompletionHints', 'create'} if __name__ == '__main__': diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py new file mode 100644 index 0000000000..bf46ea9a5d --- /dev/null +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -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. +################################################################################ +from pyflink.table import DataTypes +from pyflink.table.udf import ScalarFunction, udf +from pyflink.testing import source_sink_utils +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase + + +class UserDefinedFunctionTests(PyFlinkStreamTableTestCase): + + def test_scalar_function(self): + # test lambda function + self.t_env.register_function( + "add_one", udf(lambda i: i + 1, DataTypes.BIGINT(), DataTypes.BIGINT())) + + # test Python ScalarFunction + self.t_env.register_function( + "subtract_one", udf(SubtractOne(), DataTypes.BIGINT(), DataTypes.BIGINT())) + + # test Python function + self.t_env.register_function("add", add) + + # test callable function + self.t_env.register_function( + "add_one_callable", udf(CallablePlus(), DataTypes.BIGINT(), DataTypes.BIGINT())) + + def partial_func(col, param): + return col + param + + # test partial function + import functools + self.t_env.register_function( + "add_one_partial", + udf(functools.partial(partial_func, param=1), DataTypes.BIGINT(), DataTypes.BIGINT())) + + table_sink = source_sink_utils.TestAppendSink( + ['a', 'b', 'c', 'd', 'e'], + [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.BIGINT(), + DataTypes.BIGINT()]) + self.t_env.register_table_sink("Results", table_sink) + + t = self.t_env.from_elements([(1, 2, 3), (2, 5, 6), (3, 1, 9)], ['a', 'b', 'c']) + t.where("add_one(b) <= 3") \ + .select("add_one(a), subtract_one(b), add(a, c), add_one_callable(a), " + "add_one_partial(a)") \ + .insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, ["2,1,4,2,2", "4,0,12,4,4"]) + + def test_chaining_scalar_function(self): + self.t_env.register_function( + "add_one", udf(lambda i: i + 1, DataTypes.BIGINT(), DataTypes.BIGINT())) + self.t_env.register_function( + "subtract_one", udf(SubtractOne(), DataTypes.BIGINT(), DataTypes.BIGINT())) + self.t_env.register_function("add", add) + + table_sink = source_sink_utils.TestAppendSink(['a'], [DataTypes.BIGINT()]) + self.t_env.register_table_sink("Results", table_sink) + + t = self.t_env.from_elements([(1, 2), (2, 5), (3, 1)], ['a', 'b']) + t.select("add(add_one(a), subtract_one(b))") \ + .insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, ["3", "7", "4"]) + + def test_udf_in_join_condition(self): + t1 = self.t_env.from_elements([(2, "Hi")], ['a', 'b']) + t2 = self.t_env.from_elements([(2, "Flink")], ['c', 'd']) + + self.t_env.register_function("f", udf(lambda i: i, DataTypes.BIGINT(), DataTypes.BIGINT())) + + table_sink = source_sink_utils.TestAppendSink( + ['a', 'b', 'c', 'd'], + [DataTypes.BIGINT(), DataTypes.STRING(), DataTypes.BIGINT(), DataTypes.STRING()]) + self.t_env.register_table_sink("Results", table_sink) + + t1.join(t2).where("f(a) = c").insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, ["2,Hi,2,Flink"]) + + def test_udf_in_join_condition_2(self): + t1 = self.t_env.from_elements([(1, "Hi"), (2, "Hi")], ['a', 'b']) + t2 = self.t_env.from_elements([(2, "Flink")], ['c', 'd']) + + self.t_env.register_function("f", udf(lambda i: i, DataTypes.BIGINT(), DataTypes.BIGINT())) + + table_sink = source_sink_utils.TestAppendSink( + ['a', 'b', 'c', 'd'], + [DataTypes.BIGINT(), DataTypes.STRING(), DataTypes.BIGINT(), DataTypes.STRING()]) + self.t_env.register_table_sink("Results", table_sink) + + t1.join(t2).where("f(a) = f(c)").insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, ["2,Hi,2,Flink"]) + + def test_overwrite_builtin_function(self): + self.t_env.register_function( + "plus", udf(lambda i, j: i + j - 1, + [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT())) + + table_sink = source_sink_utils.TestAppendSink(['a'], [DataTypes.BIGINT()]) + self.t_env.register_table_sink("Results", table_sink) + + t = self.t_env.from_elements([(1, 2, 3), (2, 5, 6), (3, 1, 9)], ['a', 'b', 'c']) + t.select("plus(a, b)").insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, ["2", "6", "3"]) + + def test_open(self): + self.t_env.register_function( + "subtract", udf(Subtract(), DataTypes.BIGINT(), DataTypes.BIGINT())) + table_sink = source_sink_utils.TestAppendSink( + ['a', 'b'], [DataTypes.BIGINT(), DataTypes.BIGINT()]) + self.t_env.register_table_sink("Results", table_sink) + + t = self.t_env.from_elements([(1, 2), (2, 5), (3, 4)], ['a', 'b']) + t.select("a, subtract(b)").insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, ["1,1", "2,4", "3,3"]) + + def test_deterministic(self): + add_one = udf(lambda i: i + 1, DataTypes.BIGINT(), DataTypes.BIGINT()) + self.assertTrue(add_one._deterministic) + + add_one = udf(lambda i: i + 1, DataTypes.BIGINT(), DataTypes.BIGINT(), deterministic=False) + self.assertFalse(add_one._deterministic) + + subtract_one = udf(SubtractOne(), DataTypes.BIGINT(), DataTypes.BIGINT()) + self.assertTrue(subtract_one._deterministic) + + with self.assertRaises(ValueError, msg="Inconsistent deterministic: False and True"): + udf(SubtractOne(), DataTypes.BIGINT(), DataTypes.BIGINT(), deterministic=False) + + self.assertTrue(add._deterministic) + + @udf(input_types=DataTypes.BIGINT(), result_type=DataTypes.BIGINT(), deterministic=False) + def non_deterministic_udf(i): + return i + + self.assertFalse(non_deterministic_udf._deterministic) + + def test_name(self): + add_one = udf(lambda i: i + 1, DataTypes.BIGINT(), DataTypes.BIGINT()) + self.assertEqual("", add_one._name) + + add_one = udf(lambda i: i + 1, DataTypes.BIGINT(), DataTypes.BIGINT(), name="add_one") + self.assertEqual("add_one", add_one._name) + + subtract_one = udf(SubtractOne(), DataTypes.BIGINT(), DataTypes.BIGINT()) + self.assertEqual("SubtractOne", subtract_one._name) + + subtract_one = udf(SubtractOne(), DataTypes.BIGINT(), DataTypes.BIGINT(), + name="subtract_one") + self.assertEqual("subtract_one", subtract_one._name) + + self.assertEqual("add", add._name) + + @udf(input_types=DataTypes.BIGINT(), result_type=DataTypes.BIGINT(), name="named") + def named_udf(i): + return i + + self.assertEqual("named", named_udf._name) + + def test_abc(self): + class UdfWithoutEval(ScalarFunction): + def open(self, function_context): + pass + + with self.assertRaises( + TypeError, + msg="Can't instantiate abstract class UdfWithoutEval with abstract methods eval"): + UdfWithoutEval() + + def test_invalid_udf(self): + class Plus(object): + def eval(self, col): + return col + 1 + + with self.assertRaises( + TypeError, + msg="Invalid function: not a function or callable (__call__ is not defined)"): + # test non-callable function + self.t_env.register_function( + "non-callable-udf", udf(Plus(), DataTypes.BIGINT(), DataTypes.BIGINT())) + + +@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT()) +def add(i, j): + return i + j + + +class SubtractOne(ScalarFunction): + + def eval(self, i): + return i - 1 + + +class Subtract(ScalarFunction): + + def __init__(self): + self.subtracted_value = 0 + + def open(self, function_context): + self.subtracted_value = 1 + + def eval(self, i): + return i - self.subtracted_value + + +class CallablePlus(object): + + def __call__(self, col): + return col + 1 + + +if __name__ == '__main__': + import unittest + + try: + import xmlrunner + testRunner = xmlrunner.XMLTestRunner(output='target/test-reports') + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/flink-python/pyflink/table/types.py b/flink-python/pyflink/table/types.py index dbf92f88bb..09186240a6 100644 --- a/flink-python/pyflink/table/types.py +++ b/flink-python/pyflink/table/types.py @@ -2005,7 +2005,7 @@ class Row(tuple): return "Row(%s)" % ", ".join("%s=%r" % (k, v) for k, v in zip(self._fields, tuple(self))) else: - return "" % ", ".join(self) + return "" % ", ".join("%r" % field for field in self) _acceptable_types = { diff --git a/flink-python/pyflink/table/udf.py b/flink-python/pyflink/table/udf.py new file mode 100644 index 0000000000..97ac391070 --- /dev/null +++ b/flink-python/pyflink/table/udf.py @@ -0,0 +1,229 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +import abc +import collections +import functools +import inspect +import sys + +from pyflink.java_gateway import get_gateway +from pyflink.table.types import DataType, _to_java_type +from pyflink.util import utils + +__all__ = ['FunctionContext', 'ScalarFunction', 'udf'] + + +if sys.version_info >= (3, 4): + ABC = abc.ABC +else: + ABC = abc.ABCMeta('ABC', (), {}) + + +class FunctionContext(object): + """ + Used to obtain global runtime information about the context in which the + user-defined function is executed. The information includes the metric group, + and global job parameters, etc. + """ + pass + + +class UserDefinedFunction(ABC): + """ + Base interface for user-defined function. + """ + + def open(self, function_context): + """ + Initialization method for the function. It is called before the actual working methods + and thus suitable for one time setup work. + + :param function_context: the context of the function + :type function_context: FunctionContext + """ + pass + + def close(self): + """ + Tear-down method for the user code. It is called after the last call to the main + working methods. + """ + pass + + def is_deterministic(self): + """ + Returns information about the determinism of the function's results. + It returns true if and only if a call to this function is guaranteed to + always return the same result given the same parameters. true is assumed by default. + If the function is not pure functional like random(), date(), now(), + this method must return false. + + :return: the determinism of the function's results. + :rtype: bool + """ + return True + + +class ScalarFunction(UserDefinedFunction): + """ + Base interface for user-defined scalar function. A user-defined scalar functions maps zero, one, + or multiple scalar values to a new scalar value. + """ + + @abc.abstractmethod + def eval(self, *args): + """ + Method which defines the logic of the scalar function. + """ + pass + + +class DelegatingScalarFunction(ScalarFunction): + """ + Helper scalar function implementation for lambda expression and python function. It's for + internal use only. + """ + + def __init__(self, func): + self.func = func + + def eval(self, *args): + return self.func(*args) + + +class UserDefinedFunctionWrapper(object): + """ + Wrapper for Python user-defined function. It handles things like converting lambda + functions to user-defined functions, creating the Java user-defined function representation, + etc. It's for internal use only. + """ + + def __init__(self, func, input_types, result_type, deterministic=None, name=None): + if inspect.isclass(func) or ( + not isinstance(func, UserDefinedFunction) and not callable(func)): + raise TypeError( + "Invalid function: not a function or callable (__call__ is not defined): {0}" + .format(type(func))) + + if not isinstance(input_types, collections.Iterable): + input_types = [input_types] + + for input_type in input_types: + if not isinstance(input_type, DataType): + raise TypeError( + "Invalid input_type: input_type should be DataType but contains {}".format( + input_type)) + + if not isinstance(result_type, DataType): + raise TypeError( + "Invalid returnType: returnType should be DataType but is {}".format(result_type)) + + self._func = func + self._input_types = input_types + self._result_type = result_type + self._judf_placeholder = None + self._name = name or ( + func.__name__ if hasattr(func, '__name__') else func.__class__.__name__) + + if deterministic is not None and isinstance(func, UserDefinedFunction) and deterministic \ + != func.is_deterministic(): + raise ValueError("Inconsistent deterministic: {} and {}".format( + deterministic, func.is_deterministic())) + + # default deterministic is True + self._deterministic = deterministic if deterministic is not None else ( + func.is_deterministic() if isinstance(func, UserDefinedFunction) else True) + + @property + def _judf(self): + if self._judf_placeholder is None: + self._judf_placeholder = self._create_judf() + return self._judf_placeholder + + def _create_judf(self): + func = self._func + if not isinstance(self._func, UserDefinedFunction): + func = DelegatingScalarFunction(self._func) + + import cloudpickle + serialized_func = cloudpickle.dumps(func) + + gateway = get_gateway() + j_input_types = utils.to_jarray(gateway.jvm.TypeInformation, + [_to_java_type(i) for i in self._input_types]) + j_result_type = _to_java_type(self._result_type) + return gateway.jvm.org.apache.flink.table.util.python.PythonTableUtils \ + .createPythonScalarFunction(self._name, + bytearray(serialized_func), + j_input_types, + j_result_type, + self._deterministic, + _get_python_env()) + + +# TODO: support to configure the python execution environment +def _get_python_env(): + gateway = get_gateway() + exec_type = gateway.jvm.org.apache.flink.table.functions.python.PythonEnv.ExecType.PROCESS + return gateway.jvm.org.apache.flink.table.functions.python.PythonEnv(exec_type) + + +def _create_udf(f, input_types, result_type, deterministic, name): + return UserDefinedFunctionWrapper(f, input_types, result_type, deterministic, name) + + +def udf(f=None, input_types=None, result_type=None, deterministic=None, name=None): + """ + Helper method for creating a user-defined function. + + Example: + :: + + >>> add_one = udf(lambda i: i + 1, DataTypes.BIGINT(), DataTypes.BIGINT()) + + >>> @udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], + ... result_type=DataTypes.BIGINT()) + ... def add(i, j): + ... return i + j + + >>> class SubtractOne(ScalarFunction): + ... def eval(self, i): + ... return i - 1 + >>> subtract_one = udf(SubtractOne(), DataTypes.BIGINT(), DataTypes.BIGINT()) + + :param f: lambda function or user-defined function. + :type f: function or UserDefinedFunction or type + :param input_types: the input data types. + :type input_types: list[DataType] or DataType + :param result_type: the result data type. + :type result_type: DataType + :param name: the function name. + :type name: str + :param deterministic: the determinism of the function's results. True if and only if a call to + this function is guaranteed to always return the same result given the + same parameters. (default True) + :type deterministic: bool + :return: UserDefinedFunctionWrapper or function. + :rtype: UserDefinedFunctionWrapper or function + """ + # decorator + if f is None: + return functools.partial(_create_udf, input_types=input_types, result_type=result_type, + deterministic=deterministic, name=name) + else: + return _create_udf(f, input_types, result_type, deterministic, name) diff --git a/flink-python/pyflink/testing/test_case_utils.py b/flink-python/pyflink/testing/test_case_utils.py index c1d484ed01..21d3f09e62 100644 --- a/flink-python/pyflink/testing/test_case_utils.py +++ b/flink-python/pyflink/testing/test_case_utils.py @@ -27,11 +27,10 @@ from abc import abstractmethod from py4j.java_gateway import JavaObject from py4j.protocol import Py4JJavaError +from pyflink import gen_protos from pyflink.table.sources import CsvTableSource - from pyflink.dataset import ExecutionEnvironment from pyflink.datastream import StreamExecutionEnvironment - from pyflink.find_flink_home import _find_flink_home from pyflink.table import BatchTableEnvironment, StreamTableEnvironment from pyflink.java_gateway import get_gateway @@ -76,6 +75,8 @@ class PyFlinkTestCase(unittest.TestCase): def setUpClass(cls): cls.tempdir = tempfile.mkdtemp() + gen_protos.generate_proto_files() + os.environ["FLINK_TESTING"] = "1" _find_flink_home() diff --git a/flink-python/setup.py b/flink-python/setup.py index 78a6edd6be..f5b26da98e 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -23,6 +23,12 @@ import sys from shutil import copytree, copy, rmtree from setuptools import setup +from setuptools.command.install import install +from setuptools.command.build_py import build_py +from setuptools.command.develop import develop +from setuptools.command.egg_info import egg_info +from setuptools.command.sdist import sdist +from setuptools.command.test import test if sys.version_info < (2, 7): print("Python versions prior to 2.7 are not supported for PyFlink.", @@ -62,6 +68,24 @@ README_FILE_TEMP_PATH = os.path.join("pyflink", "README.txt") in_flink_source = os.path.isfile("../flink-java/src/main/java/org/apache/flink/api/java/" "ExecutionEnvironment.java") + +# We must generate protos after setup_requires are installed. +def generate_protos_first(original_cmd): + try: + # pylint: disable=wrong-import-position + from pyflink import gen_protos + + class cmd(original_cmd, object): + def run(self): + gen_protos.generate_proto_files() + super(cmd, self).run() + return cmd + except ImportError: + import warnings + warnings.warn("Could not import gen_protos, skipping proto generation.") + return original_cmd + + try: if in_flink_source: @@ -184,7 +208,8 @@ run sdist. license='https://www.apache.org/licenses/LICENSE-2.0', author='Flink Developers', author_email='dev@flink.apache.org', - install_requires=['py4j==0.10.8.1', 'python-dateutil', 'apache-beam==2.15.0'], + install_requires=['py4j==0.10.8.1', 'python-dateutil', 'apache-beam==2.15.0', + 'cloudpickle==1.2.2'], tests_require=['pytest==4.4.1'], description='Apache Flink Python API', long_description=long_description, @@ -195,7 +220,15 @@ run sdist. 'Programming Language :: Python :: 2.7', 'Programming Language :: Python :: 3.5', 'Programming Language :: Python :: 3.6', - 'Programming Language :: Python :: 3.7'] + 'Programming Language :: Python :: 3.7'], + cmdclass={ + 'build_py': generate_protos_first(build_py), + 'develop': generate_protos_first(develop), + 'egg_info': generate_protos_first(egg_info), + 'sdist': generate_protos_first(sdist), + 'test': generate_protos_first(test), + 'install': generate_protos_first(install), + }, ) finally: if in_flink_source: diff --git a/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java index 73c753544a..53e6ef94fb 100644 --- a/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java @@ -20,6 +20,7 @@ package org.apache.flink.python; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; import org.apache.flink.table.functions.python.PythonEnv; @@ -50,8 +51,6 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; import java.util.Random; /** @@ -280,14 +279,15 @@ public abstract class AbstractPythonFunctionRunner implements PythonFun */ protected RunnerApi.Environment createPythonExecutionEnvironment() { if (pythonEnv.getExecType() == PythonEnv.ExecType.PROCESS) { - final Map env = new HashMap<>(2); - env.put("python", pythonEnv.getPythonExec()); + String flinkHomePath = System.getenv(ConfigConstants.ENV_FLINK_HOME_DIR); + String pythonWorkerCommand = + flinkHomePath + File.separator + "bin" + File.separator + "pyflink-udf-runner.sh"; return Environments.createProcessEnvironment( "", "", - pythonEnv.getPythonWorkerCmd(), - env); + pythonWorkerCommand, + null); } else { throw new UnsupportedOperationException(String.format( "Execution type '%s' is not supported.", pythonEnv.getExecType())); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractPythonScalarFunctionOperator.java index 4ce8a9ce7e..bbbc9be964 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractPythonScalarFunctionOperator.java @@ -87,9 +87,9 @@ public abstract class AbstractPythonScalarFunctionOperator inputType.getFields().get(i)) .collect(Collectors.toList())); - udfOutputType = new RowType(outputType.getFields().subList(forwardedFieldCnt, outputType.getFieldCount())); + udfOutputType = new RowType(outputType.getFields().subList(forwardedFields.length, outputType.getFieldCount())); super.open(); } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperator.java index 2f53808f5b..4274ad550f 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperator.java @@ -37,6 +37,9 @@ import org.apache.flink.util.Collector; import org.apache.beam.sdk.fn.data.FnDataReceiver; +import java.util.Arrays; +import java.util.stream.Collectors; + /** * The Python {@link ScalarFunction} operator for the blink planner. */ @@ -71,8 +74,8 @@ public class BaseRowPythonScalarFunctionOperator RowType inputType, RowType outputType, int[] udfInputOffsets, - int forwardedFieldCnt) { - super(scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFieldCnt); + int[] forwardedFields) { + super(scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFields); } @Override @@ -133,18 +136,16 @@ public class BaseRowPythonScalarFunctionOperator } private Projection createForwardedFieldProjection() { - final int[] fields = new int[forwardedFieldCnt]; - for (int i = 0; i < fields.length; i++) { - fields[i] = i; - } - - final RowType forwardedFieldType = new RowType(inputType.getFields().subList(0, forwardedFieldCnt)); + final RowType forwardedFieldType = new RowType( + Arrays.stream(forwardedFields) + .mapToObj(i -> inputType.getFields().get(i)) + .collect(Collectors.toList())); final GeneratedProjection generatedProjection = ProjectionCodeGenerator.generateProjection( CodeGeneratorContext.apply(new TableConfig()), "ForwardedFieldProjection", inputType, forwardedFieldType, - fields); + forwardedFields); // noinspection unchecked return generatedProjection.newInstance(Thread.currentThread().getContextClassLoader()); } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperator.java index 9961b16548..489c894476 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperator.java @@ -36,6 +36,8 @@ import org.apache.flink.util.Collector; import org.apache.beam.sdk.fn.data.FnDataReceiver; +import java.util.Arrays; + /** * The Python {@link ScalarFunction} operator for the legacy planner. */ @@ -59,8 +61,8 @@ public class PythonScalarFunctionOperator extends AbstractPythonScalarFunctionOp RowType inputType, RowType outputType, int[] udfInputOffsets, - int forwardedFieldCnt) { - super(scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFieldCnt); + int[] forwardedFields) { + super(scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFields); } @Override @@ -69,8 +71,8 @@ public class PythonScalarFunctionOperator extends AbstractPythonScalarFunctionOp this.cRowWrapper = new StreamRecordCRowWrappingCollector(output); CRowTypeInfo forwardedInputTypeInfo = new CRowTypeInfo(new RowTypeInfo( - inputType.getFields().stream() - .limit(forwardedFieldCnt) + Arrays.stream(forwardedFields) + .mapToObj(i -> inputType.getFields().get(i)) .map(RowType.RowField::getType) .map(TypeConversions::fromLogicalToDataType) .map(TypeConversions::fromDataTypeToLegacyInfo) @@ -80,7 +82,7 @@ public class PythonScalarFunctionOperator extends AbstractPythonScalarFunctionOp @Override public void bufferInput(CRow input) { - CRow forwardedFieldsRow = new CRow(getForwardedRow(input.row()), input.change()); + CRow forwardedFieldsRow = new CRow(Row.project(input.row(), forwardedFields), input.change()); if (getExecutionConfig().isObjectReuseEnabled()) { forwardedFieldsRow = forwardedInputSerializer.copy(forwardedFieldsRow); } @@ -115,14 +117,6 @@ public class PythonScalarFunctionOperator extends AbstractPythonScalarFunctionOp getContainingTask().getEnvironment().getTaskManagerInfo().getTmpDirectories()); } - private Row getForwardedRow(Row input) { - Row row = new Row(forwardedFieldCnt); - for (int i = 0; i < row.getArity(); i++) { - row.setField(i, input.getField(i)); - } - return row; - } - /** * The collector is used to convert a {@link Row} to a {@link CRow}. */ diff --git a/flink-python/src/test/java/org/apache/flink/table/functions/python/AbstractPythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/functions/python/AbstractPythonScalarFunctionRunnerTest.java index c53f69a06b..4c28d6a851 100644 --- a/flink-python/src/test/java/org/apache/flink/table/functions/python/AbstractPythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/functions/python/AbstractPythonScalarFunctionRunnerTest.java @@ -121,7 +121,7 @@ public abstract class AbstractPythonScalarFunctionRunnerTest { @Override public PythonEnv getPythonEnv() { - return new PythonEnv("", "", PythonEnv.ExecType.PROCESS); + return new PythonEnv(PythonEnv.ExecType.PROCESS); } } } diff --git a/flink-python/src/test/java/org/apache/flink/table/functions/python/BaseRowPythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/functions/python/BaseRowPythonScalarFunctionRunnerTest.java index 02f5b7462c..dacd5092b5 100644 --- a/flink-python/src/test/java/org/apache/flink/table/functions/python/BaseRowPythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/functions/python/BaseRowPythonScalarFunctionRunnerTest.java @@ -116,7 +116,7 @@ public class BaseRowPythonScalarFunctionRunnerTest extends AbstractPythonScalarF // ignore the execution results }; - final PythonEnv pythonEnv = new PythonEnv("", "", PythonEnv.ExecType.PROCESS); + final PythonEnv pythonEnv = new PythonEnv(PythonEnv.ExecType.PROCESS); return new BaseRowPythonScalarFunctionRunner( "testPythonRunner", diff --git a/flink-python/src/test/java/org/apache/flink/table/functions/python/PythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/functions/python/PythonScalarFunctionRunnerTest.java index 6a9398a202..212bf2a6e8 100644 --- a/flink-python/src/test/java/org/apache/flink/table/functions/python/PythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/functions/python/PythonScalarFunctionRunnerTest.java @@ -244,7 +244,7 @@ public class PythonScalarFunctionRunnerTest extends AbstractPythonScalarFunction // ignore the execution results }; - final PythonEnv pythonEnv = new PythonEnv("", "", PythonEnv.ExecType.PROCESS); + final PythonEnv pythonEnv = new PythonEnv(PythonEnv.ExecType.PROCESS); return new PythonScalarFunctionRunner( "testPythonRunner", @@ -266,7 +266,7 @@ public class PythonScalarFunctionRunnerTest extends AbstractPythonScalarFunction RowType rowType = new RowType(Collections.singletonList(new RowType.RowField("f1", new BigIntType()))); - final PythonEnv pythonEnv = new PythonEnv("", "", PythonEnv.ExecType.PROCESS); + final PythonEnv pythonEnv = new PythonEnv(PythonEnv.ExecType.PROCESS); return new PythonScalarFunctionRunnerTestHarness( "testPythonRunner", diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperatorTest.java index 35288cbfa7..93704d2820 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/BaseRowPythonScalarFunctionOperatorTest.java @@ -52,13 +52,13 @@ public class BaseRowPythonScalarFunctionOperatorTest RowType inputType, RowType outputType, int[] udfInputOffsets, - int forwardedFieldCnt) { + int[] forwardedFields) { return new PassThroughPythonScalarFunctionOperator( scalarFunctions, inputType, outputType, udfInputOffsets, - forwardedFieldCnt + forwardedFields ); } @@ -83,8 +83,8 @@ public class BaseRowPythonScalarFunctionOperatorTest RowType inputType, RowType outputType, int[] udfInputOffsets, - int forwardedFieldCnt) { - super(scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFieldCnt); + int[] forwardedFields) { + super(scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFields); } @Override diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTest.java index d9a854e955..57e7497e05 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTest.java @@ -41,9 +41,9 @@ public class PythonScalarFunctionOperatorTest extends PythonScalarFunctionOperat RowType inputType, RowType outputType, int[] udfInputOffsets, - int forwardedFieldCnt) { + int[] forwardedFields) { return new PassThroughPythonScalarFunctionOperator( - scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFieldCnt); + scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFields); } @Override @@ -63,8 +63,8 @@ public class PythonScalarFunctionOperatorTest extends PythonScalarFunctionOperat RowType inputType, RowType outputType, int[] udfInputOffsets, - int forwardedFieldCnt) { - super(scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFieldCnt); + int[] forwardedFields) { + super(scalarFunctions, inputType, outputType, udfInputOffsets, forwardedFields); } @Override diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTestBase.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTestBase.java index a4c52a8375..195fc77ca7 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTestBase.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/PythonScalarFunctionOperatorTestBase.java @@ -204,7 +204,7 @@ public abstract class PythonScalarFunctionOperatorTestBase(operator); @@ -215,7 +215,7 @@ public abstract class PythonScalarFunctionOperatorTestBase + s"${primitiveTypeTermForTypeInfo(inputType)} in$index" + }.mkString(", ") + + val encodingUtilsTypeTerm = classOf[EncodingUtils].getCanonicalName + val typeInfoTypeTerm = classOf[TypeInformation[_]].getCanonicalName + val inputTypesCode = inputTypes.map(EncodingUtils.encodeObjectToString).map { inputType => + s""" + |($typeInfoTypeTerm) $encodingUtilsTypeTerm.decodeStringToObject( + | "$inputType", $typeInfoTypeTerm.class) + |""".stripMargin + }.mkString(", ") + + val encodedResultType = EncodingUtils.encodeObjectToString(resultType) + val encodedScalarFunction = EncodingUtils.encodeBytesToBase64(serializedScalarFunction) + val encodedPythonEnv = EncodingUtils.encodeObjectToString(pythonEnv) + val pythonEnvTypeTerm = classOf[PythonEnv].getCanonicalName + + val funcCode = j""" + |public class $funcName extends ${classOf[ScalarFunction].getCanonicalName} + | implements ${classOf[PythonFunction].getCanonicalName} { + | + | private static final long serialVersionUID = 1L; + | + | public $resultTypeTerm eval($inputParamCode) { + | return $defaultResultValue; + | } + | + | @Override + | public $typeInfoTypeTerm[] getParameterTypes(Class[] signature) { + | return new $typeInfoTypeTerm[]{$inputTypesCode}; + | } + | + | @Override + | public $typeInfoTypeTerm getResultType(Class[] signature) { + | return ($typeInfoTypeTerm) $encodingUtilsTypeTerm.decodeStringToObject( + | "$encodedResultType", $typeInfoTypeTerm.class); + | } + | + | @Override + | public ${classOf[FunctionLanguage].getCanonicalName} getLanguage() { + | return ${classOf[FunctionLanguage].getCanonicalName}.PYTHON; + | } + | + | @Override + | public byte[] getSerializedPythonFunction() { + | return $encodingUtilsTypeTerm.decodeBase64ToBytes("$encodedScalarFunction"); + | } + | + | @Override + | public $pythonEnvTypeTerm getPythonEnv() { + | return ($pythonEnvTypeTerm) $encodingUtilsTypeTerm.decodeStringToObject( + | "$encodedPythonEnv", $pythonEnvTypeTerm.class); + | } + | + | @Override + | public boolean isDeterministic() { + | return $deterministic; + | } + | + | @Override + | public String toString() { + | return "$name"; + | } + |} + |""".stripMargin + + val clazz = compile( + Thread.currentThread().getContextClassLoader, + funcName, + funcCode) + clazz.newInstance().asInstanceOf[ScalarFunction] + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCalc.scala new file mode 100644 index 0000000000..04f4c20a75 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCalc.scala @@ -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.table.plan.nodes + +import org.apache.calcite.rex.{RexCall, RexInputRef, RexLiteral, RexNode} +import org.apache.flink.table.functions.FunctionLanguage +import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo, SimplePythonFunction} +import org.apache.flink.table.functions.utils.ScalarSqlFunction + +import scala.collection.JavaConversions._ +import scala.collection.mutable + +trait CommonPythonCalc { + + private[flink] def extractPythonScalarFunctionInfos( + rexCalls: Array[RexCall]): (Array[Int], Array[PythonFunctionInfo]) = { + // using LinkedHashMap to keep the insert order + val inputNodes = new mutable.LinkedHashMap[RexNode, Integer]() + val pythonFunctionInfos = rexCalls.map(createPythonScalarFunctionInfo(_, inputNodes)) + + val udfInputOffsets = inputNodes.toArray.map(_._1).map { + case inputRef: RexInputRef => inputRef.getIndex + case _: RexLiteral => throw new Exception( + "Constants cannot be used as parameters of Python UDF for now. " + + "It will be supported in FLINK-14208") + } + (udfInputOffsets, pythonFunctionInfos) + } + + private[flink] def createPythonScalarFunctionInfo( + rexCall: RexCall, + inputNodes: mutable.Map[RexNode, Integer]): PythonFunctionInfo = rexCall.getOperator match { + case sfc: ScalarSqlFunction if sfc.getScalarFunction.getLanguage == FunctionLanguage.PYTHON => + val inputs = new mutable.ArrayBuffer[AnyRef]() + rexCall.getOperands.foreach { + case pythonRexCall: RexCall if pythonRexCall.getOperator.asInstanceOf[ScalarSqlFunction] + .getScalarFunction.getLanguage == FunctionLanguage.PYTHON => + // Continuous Python UDFs can be chained together + val argPythonInfo = createPythonScalarFunctionInfo(pythonRexCall, inputNodes) + inputs.append(argPythonInfo) + + case argNode: RexNode => + // For input arguments of RexInputRef, it's replaced with an offset into the input row + inputNodes.get(argNode) match { + case Some(existing) => inputs.append(existing) + case None => + val inputOffset = Integer.valueOf(inputNodes.size) + inputs.append(inputOffset) + inputNodes.put(argNode, inputOffset) + } + } + + // Extracts the necessary information for Python function execution, such as + // the serialized Python function, the Python env, etc + val pythonFunction = new SimplePythonFunction( + sfc.getScalarFunction.asInstanceOf[PythonFunction].getSerializedPythonFunction, + sfc.getScalarFunction.asInstanceOf[PythonFunction].getPythonEnv) + new PythonFunctionInfo(pythonFunction, inputs.toArray) + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala index fbb7be14bc..2a8dce24ef 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala @@ -21,12 +21,25 @@ package org.apache.flink.table.plan.nodes.datastream import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.core.Calc -import org.apache.calcite.rex.RexProgram +import org.apache.calcite.rex.{RexCall, RexInputRef, RexLocalRef, RexNode, RexProgram} +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.streaming.api.operators.OneInputStreamOperator import org.apache.flink.table.api.StreamQueryConfig +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.codegen.FunctionCodeGenerator +import org.apache.flink.table.functions.python.PythonFunctionInfo +import org.apache.flink.table.plan.nodes.CommonPythonCalc +import org.apache.flink.table.plan.nodes.datastream.DataStreamPythonCalc.PYTHON_SCALAR_FUNCTION_OPERATOR_NAME import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.planner.StreamPlanner -import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.runtime.CRowProcessRunner +import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} +import org.apache.flink.table.types.logical.RowType +import org.apache.flink.table.types.utils.TypeConversions + +import scala.collection.JavaConversions._ /** * RelNode for Python ScalarFunctions. @@ -46,7 +59,8 @@ class DataStreamPythonCalc( inputSchema, schema, calcProgram, - ruleDescription) { + ruleDescription) + with CommonPythonCalc { override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = { new DataStreamPythonCalc( @@ -59,10 +73,111 @@ class DataStreamPythonCalc( ruleDescription) } + private lazy val pythonRexCalls = calcProgram.getProjectList + .map(calcProgram.expandLocalRef) + .filter(_.isInstanceOf[RexCall]) + .map(_.asInstanceOf[RexCall]) + .toArray + + private lazy val forwardedFields: Array[Int] = calcProgram.getProjectList + .map(calcProgram.expandLocalRef) + .filter(_.isInstanceOf[RexInputRef]) + .map(_.asInstanceOf[RexInputRef].getIndex) + .toArray + + private lazy val (pythonUdfInputOffsets, pythonFunctionInfos) = + extractPythonScalarFunctionInfos(pythonRexCalls) + + private lazy val resultProjectList: Array[RexNode] = { + var idx = 0 + calcProgram.getProjectList + .map(calcProgram.expandLocalRef) + .map { + case pythonCall: RexCall => + val inputRef = new RexInputRef(forwardedFields.length + idx, pythonCall.getType) + idx += 1 + inputRef + case node => node + } + .toArray + } + override def translateToPlan( planner: StreamPlanner, queryConfig: StreamQueryConfig): DataStream[CRow] = { - // Will add the implementation in FLINK-14018 as it's not testable for now. - null + val config = planner.getConfig + + val inputDataStream = + getInput.asInstanceOf[DataStreamRel].translateToPlan(planner, queryConfig) + + val inputParallelism = inputDataStream.getParallelism + + val pythonOperatorResultTypeInfo = new RowTypeInfo( + forwardedFields.map(inputSchema.fieldTypeInfos.get(_)) ++ + pythonRexCalls.map(node => FlinkTypeFactory.toTypeInfo(node.getType)): _*) + + // Constructs the Python operator + val pythonOperatorInputRowType = TypeConversions.fromLegacyInfoToDataType( + inputSchema.typeInfo).getLogicalType.asInstanceOf[RowType] + val pythonOperatorOutputRowType = TypeConversions.fromLegacyInfoToDataType( + pythonOperatorResultTypeInfo).getLogicalType.asInstanceOf[RowType] + val pythonOperator = getPythonScalarFunctionOperator( + pythonOperatorInputRowType, pythonOperatorOutputRowType, pythonUdfInputOffsets) + + val pythonDataStream = inputDataStream + .transform( + calcOpName(calcProgram, getExpressionString), + CRowTypeInfo(pythonOperatorResultTypeInfo), + pythonOperator) + // keep parallelism to ensure order of accumulate and retract messages + .setParallelism(inputParallelism) + + val generator = new FunctionCodeGenerator( + config, false, pythonOperatorResultTypeInfo) + + val genFunction = generateFunction( + generator, + ruleDescription, + schema, + resultProjectList, + None, + config, + classOf[ProcessFunction[CRow, CRow]]) + + val processFunc = new CRowProcessRunner( + genFunction.name, + genFunction.code, + CRowTypeInfo(schema.typeInfo)) + + pythonDataStream + .process(processFunc) + .name(calcOpName(calcProgram, getExpressionString)) + // keep parallelism to ensure order of accumulate and retract messages + .setParallelism(inputParallelism) + } + + private[flink] def getPythonScalarFunctionOperator( + inputRowType: RowType, + outputRowType: RowType, + udfInputOffsets: Array[Int]) = { + val clazz = Class.forName(PYTHON_SCALAR_FUNCTION_OPERATOR_NAME) + val ctor = clazz.getConstructor( + classOf[Array[PythonFunctionInfo]], + classOf[RowType], + classOf[RowType], + classOf[Array[Int]], + classOf[Array[Int]]) + ctor.newInstance( + pythonFunctionInfos, + inputRowType, + outputRowType, + udfInputOffsets, + forwardedFields) + .asInstanceOf[OneInputStreamOperator[CRow, CRow]] } } + +object DataStreamPythonCalc { + val PYTHON_SCALAR_FUNCTION_OPERATOR_NAME = + "org.apache.flink.table.runtime.operators.python.PythonScalarFunctionOperator" +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCalcRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCalcRule.scala index e164c09910..7c618dceb7 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCalcRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCalcRule.scala @@ -55,7 +55,7 @@ class DataStreamPythonCalcRule new RowSchema(convInput.getRowType), new RowSchema(rel.getRowType), calc.getProgram, - description) + "DataStreamPythonCalcRule") } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/util/python/PythonTableUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/util/python/PythonTableUtils.scala index 094945c344..a84dad8159 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/util/python/PythonTableUtils.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/util/python/PythonTableUtils.scala @@ -31,6 +31,9 @@ import org.apache.flink.api.java.io.CollectionInputFormat import org.apache.flink.api.java.typeutils.{MapTypeInfo, ObjectArrayTypeInfo, RowTypeInfo} import org.apache.flink.core.io.InputSplit import org.apache.flink.table.api.{TableSchema, Types} +import org.apache.flink.table.codegen.PythonFunctionCodeGenerator +import org.apache.flink.table.functions.ScalarFunction +import org.apache.flink.table.functions.python.PythonEnv import org.apache.flink.table.sources.InputFormatTableSource import org.apache.flink.types.Row @@ -38,6 +41,32 @@ import scala.collection.JavaConversions._ object PythonTableUtils { + /** + * Creates a [[ScalarFunction]] for the specified Python ScalarFunction. + * + * @param funcName class name of the user-defined function. Must be a valid Java class identifier + * @param serializedScalarFunction serialized Python scalar function + * @param inputTypes input data types + * @param resultType expected result type + * @param deterministic the determinism of the function's results + * @param pythonEnv the Python execution environment + * @return A generated Java ScalarFunction representation for the specified Python ScalarFunction + */ + def createPythonScalarFunction( + funcName: String, + serializedScalarFunction: Array[Byte], + inputTypes: Array[TypeInformation[_]], + resultType: TypeInformation[_], + deterministic: Boolean, + pythonEnv: PythonEnv): ScalarFunction = + PythonFunctionCodeGenerator.generateScalarFunction( + funcName, + serializedScalarFunction, + inputTypes, + resultType, + deterministic, + pythonEnv) + /** * Wrap the unpickled python data with an InputFormat. It will be passed to * PythonInputFormatTableSource later. diff --git a/pom.xml b/pom.xml index 5a8b598391..832e298d85 100644 --- a/pom.xml +++ b/pom.xml @@ -1422,6 +1422,7 @@ under the License. flink-python/lib/** flink-python/dev/download/** flink-python/docs/_build/** + flink-python/pyflink/fn_execution/*_pb2.py -- Gitee From fc4a1df1ad48224ae846fbbdeeb21b4b7b164e02 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 26 Sep 2019 12:05:53 +0800 Subject: [PATCH 084/268] [FLINK-14018][python] Package cloudpickle in flink for ease of use for Flink Python users This closes #9766. --- NOTICE | 1 + NOTICE-binary | 1 + flink-python/README.md | 2 +- flink-python/bin/pyflink-shell.sh | 5 +-- flink-python/bin/pyflink-udf-runner.sh | 6 ++++ flink-python/lib/cloudpickle-1.2.2-src.zip | Bin 0 -> 23425 bytes flink-python/lib/cloudpickle-LICENSE.txt | 32 ++++++++++++++++++ .../src/main/resources/META-INF/NOTICE | 1 + .../META-INF/licenses/LICENSE.cloudpickle | 32 ++++++++++++++++++ licenses-binary/LICENSE.cloudpickle | 32 ++++++++++++++++++ licenses/LICENSE.cloudpickle | 32 ++++++++++++++++++ 11 files changed, 141 insertions(+), 3 deletions(-) create mode 100644 flink-python/lib/cloudpickle-1.2.2-src.zip create mode 100644 flink-python/lib/cloudpickle-LICENSE.txt create mode 100644 flink-python/src/main/resources/META-INF/licenses/LICENSE.cloudpickle create mode 100644 licenses-binary/LICENSE.cloudpickle create mode 100644 licenses/LICENSE.cloudpickle diff --git a/NOTICE b/NOTICE index 88121c1d7e..30aa331903 100644 --- a/NOTICE +++ b/NOTICE @@ -33,6 +33,7 @@ This project bundles the following dependencies under the BSD license. See bundled license files for details. - d3:3.5.12 +- cloudpickle:1.2.2 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. diff --git a/NOTICE-binary b/NOTICE-binary index 122242c180..afea5236f2 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -8035,6 +8035,7 @@ See bundled license files for details - com.google.protobuf:protobuf-java:3.7.1 - com.google.protobuf:protobuf-java-util:3.7.1 - com.google.auth:google-auth-library-credentials:0.13.0 +- cloudpickle:1.2.2 This project bundles the following dependencies under the MIT license. (https://opensource.org/licenses/MIT) See bundled license files for details. diff --git a/flink-python/README.md b/flink-python/README.md index 7f8c936144..db49150381 100644 --- a/flink-python/README.md +++ b/flink-python/README.md @@ -27,4 +27,4 @@ We can enter the directory where this README.md file is located and run test cas ## Python Requirements -PyFlink depends on Py4J (currently version 0.10.8.1). +PyFlink depends on Py4J (currently version 0.10.8.1) and CloudPickle (currently version 1.2.2). diff --git a/flink-python/bin/pyflink-shell.sh b/flink-python/bin/pyflink-shell.sh index 2c83cc76af..8722e20831 100755 --- a/flink-python/bin/pyflink-shell.sh +++ b/flink-python/bin/pyflink-shell.sh @@ -34,10 +34,11 @@ PYFLINK_PYTHON="${PYFLINK_PYTHON:-"python"}" export FLINK_BIN_DIR=$FLINK_BIN_DIR export FLINK_HOME -# Add pyflink & py4j to PYTHONPATH +# Add pyflink & py4j & cloudpickle to PYTHONPATH export PYTHONPATH="$FLINK_OPT_DIR/python/pyflink.zip:$PYTHONPATH" PY4J_ZIP=`echo "$FLINK_OPT_DIR"/python/py4j-*-src.zip` -export PYTHONPATH="$PY4J_ZIP:$PYTHONPATH" +CLOUDPICKLE_ZIP=`echo "$FLINK_OPT_DIR"/python/cloudpickle-*-src.zip` +export PYTHONPATH="$PY4J_ZIP:$CLOUDPICKLE_ZIP:$PYTHONPATH" PARSER="org.apache.flink.client.python.PythonShellParser" function parse_options() { diff --git a/flink-python/bin/pyflink-udf-runner.sh b/flink-python/bin/pyflink-udf-runner.sh index 38c38cd314..edfb85c718 100755 --- a/flink-python/bin/pyflink-udf-runner.sh +++ b/flink-python/bin/pyflink-udf-runner.sh @@ -58,6 +58,12 @@ if [[ ! ${PYTHONPATH} =~ ${PY4J_ZIP} ]]; then export PYTHONPATH="$PY4J_ZIP:$PYTHONPATH" fi +# Add cloudpickle to PYTHONPATH +CLOUDPICKLE_ZIP=`echo "$FLINK_OPT_DIR"/python/cloudpickle-*-src.zip` +if [[ ! ${PYTHONPATH} =~ ${CLOUDPICKLE_ZIP} ]]; then + export PYTHONPATH="$CLOUDPICKLE_ZIP:$PYTHONPATH" +fi + log="$FLINK_LOG_DIR/flink-$FLINK_IDENT_STRING-python-udf-boot-$HOSTNAME.log" ${python} -m pyflink.fn_execution.boot $@ 2>&1 | tee -a ${log} diff --git a/flink-python/lib/cloudpickle-1.2.2-src.zip b/flink-python/lib/cloudpickle-1.2.2-src.zip new file mode 100644 index 0000000000000000000000000000000000000000..82bc04ec2740210a4fbd2c4cd1ba0947be478b07 GIT binary patch literal 23425 zcmb6AQ>-vd5G{yq+qP}nwr$(CZQHhO+qP}{+x!0K%$<8@CTB94N~+WSP%E|Sq0_1A zr63Iq0tN6tjKqLT{(o-%zY7uo4uG+Zy{n0XrLnb*DJ>%%6CD%1stP0kFbGGCuIYd7 z>H!S^2=WRD004sWUzEcC56ifi>i>j=_rJvYKcN3_xLIlL|980m10Mpwv4~1uHAP$^ z{=d`Je@^-Th^wz}X=mx8uTSUT`9D)%OL5ozFA(7WlDco~G}I6Sg6R7*YIC&|BnG=u zMfPuicVb~1CPbO+gz&>X>sSD_IrBSS2J_8rkVuZvQxwGOQB6F@UJxrK6&3Em^f2?v z#fu9&6aWV`nennt!pG!lJMiqM22Tq7R$YmDTe_Uym0(HEdA;kjB{%wN#K&enW&Y;v~sBsSR72(G__# zecpeoy1wgpBsvG&3l6E$&9i5^3X-rHwxLuL#EeSKK1J-2M&_6; zws8rVN@qrBjG9+!GG!RlWc{#*mBlU3j>t@+5Hlf;EmMq_1!|E=Q@nLkUUE3njrDa;kWCJnJYa0=uGtFtxk8JqD97#fzE+$(~Sk@(R z!0hx++W#7d%Rj0=ZpIM}QkN1XF03Wz1bznQgj%MBBrl24IUN9#b(I%^ABwDRSdIb>&uhpfwS4 zu~?EeLyI4rb1Rxhm0ENV0;5**%r=C_#3W0AA#=d!VM;O~3X*|OTN;AgkOA=$Dc4P6 zS2Kr*9>yL@k=-GRH5iDZr6Jllr3H8XVe5tLl@_unFZ+NlL)CQY2_tVos|$|tWuz&Y zt0Ep=vE7{6nNB=k_~O)@w8ylP)|os}5OLYPIbG8 z;dco$-t5QlwS(skoOcWUDYqZHj;sb__&$ud{@oca`2HD^@V$8B+B2ZsZo}p6#ewZs ze=GVkxX5h@&+r_g-)@5+{yVlifDJF=N7#*b@1=_f{3`I}zVZ_GxVd$`?_2Kmg*V)d zAF=g6dFjIMCWgU&)$Y0YWm)V;#P1z^1^2PlclF*~1g>{q#@$`gV*eMojOKnjSmEuv zZw)`P`|qHR{{_UhdKp*meQP&fZ+!GhL(wC@XFCzY&*I(dRhP>veZ<=}{%8uh*Y3OO zd{L@rTF(7IH{OE#wA@+(?&no&?qRO`?g%*K(2)6dPo)%OeM8$= zpJXUvgfLNneEl8xp7;@l6Xw1?toZK2?$Q065GDH}{#_Plim|-u3gNi2c^EL1N>%MopdS0(>a;p4p;d)42*TGG1 zBFSk#JL6gJJ*Gl9JYp6L%pb8z5%_LG2__NbZUz-m@(?UMOMDS4Mu8Qk z{Z*{Mkw@7wKq0{2y5+w|+9)O{GPmv{lp;Y#+ie=bO}#k;&FJuL)|x%L69w@jWDpk$AH7qop0HjV?LrLCn>G zCLmHK2*@8r3b2z3EWjp(3LRRfjgv9x28bD;gVe}^D2S;~tU!I4B$%nBopUynp{Lxj zQ5V;gJR|whG~pF=S9C z99!~03lR;3&!!E)Rzi1#J>dODn}qpV=*$#kB$sR$9220_48qG`PR1>tZ3SSKC6{w1 zTuNqavC4^1sV~TJoPwB&SRPbff8>U5zm4Cu#v^dye4SkMdjbDl3GxSo#OQYBKYpus z-iBcLlW&VXhi>l9&8^4_rcCoQcdqyUqB=uoyLwxFMI6rU4(|e6&U8dqH#Wgv$C>~_ zV97dqQp(-6xo;znXPy{>Y(Gt6`aYpq4-t|8I&HP-Lo(mR45lW=Dmb=;_9CeP0%+Q= zYvcJwko~ioAx1T+yB~Kt{{EN+_F;${Js7QH)aQ>mznadjKQe!O3qsQ;A2dSM0Z@vN z1WKiqes94m!+nuchxzGpI5xU*fmlJL@ABo|@jEvh|LMy3J-SKhf3Q=wX=j8qxcaGC zEukH`Y7YS4uy^H|~C$ZL5^4k_re9n}X{W zdQA)Rjqyr!^|5EA?qRYgttP$B;zN!fioBLjnPTA7K#v(@H0KLs9DkIhElcf-SOIbQ zv1HJlvhPSZ9vBX#KTc6XMf3#ehjsJAug}sli!s)J$WY4mB12A!#Gh;UWzX5sov3Lt z5QKE=7Ka(k!>Q1@n@CFW)A_pBExx2Xb+-Ls|M*mTctTBai~N(6+MP?^%_L<_Fk_-9 zd?MM?AEF>tx?%8F~dMTQCzD) z-|O>L zFPB%=hlGZ^Vve$l+r$F~Nihcy@Qk^dE!$|n<1QOcQz=~#2(omX+@N{2j<|hR!rWcz ztcEG6hjWHdhL$0}tc5V^8ZoN?IA|eJ5V%ZQ2*_MxTImpFgbiq5;3%qff-WwZaiy!O z=>P+Oe7`)Mqa`I9Q0*Zrgf=+_W_-vDIV_D5(2;`lvCi`M-ePp zg=rJ}OLZLUxI;=!ypkwV*|=mfY68tDiv{n}7saK$8z#K5z(QCgBFM3g1`wX_TgB}s z(V?jbA6*P*?66eJRQ$|p;DaKn!?kjgjZ-yjSG@|*v{KP<9Q>fNWuboSBSs!LQnp4e zFDmdMDK1ulJk~*jxAs_8sPE1pGsg*ipoI`NNB73Oh`_rTFHnj1SDe(y?h(VQBulI6 z533Z^Ep69p@l5nIy+!?U17cxFg4SFt~?JzLYFpwQs1 zp1zBi{S7OM(Yw{Y?H4f%9|1JyL~73=mQ&JyMFn-U<5beIOib8W+Q0Fz!_D+)LYJg! zX&FQ>a`u#y&`Uh#M{&w3g>UzcY^;veBa^;k=ifBWfXn%DR+T{XwNSsYO8mYcr|}U1oX=FADI1 ztC*=pSP)$iClgjpnTEMqD8h!{d7)UV&%q{jD$P5o5sGP6q-~h3j5Thx!SJ$WKaHy9 zp{5-T*@o(6(Z5b4qrAyKM%|o&DTOKTex%17>e~UZcQNlB@`YyQLMDNiZ)S*+AZF!; zcAa3DkKw&B4xL(R1%MdLX{WjB_Mt>ji@tBeZBq8{=em%~!KX$D#+D+)tu(!2$WH|y zDS#Xh4fQM#vXxQAB=}?p#?~lgAck=&){CK-$C5ahh!?R!s=OrTTdkCva_Do~8jO%wDVxfr;;mQ_Gi}}f97YG-rRi(B{(vxfi6u(uhb$5w4z6G8`Qy)t zBH}_?l`3lX3*3yM3)1jL9~HRGEXhSis=MwYSAE_e(|m||z(vj=a5n-;QA}=GGFk$k zqQnhCWE_2=C|89M#+SIw(I!4?Sr(@M=%AcY!koo4Rj0U=jV8Z|gR8hiQGUYE>Hu*) z1F;7CX7ws2_jY0E6c`eNv7Nz}p* zTCJm86AUyXSd;BK9)`(nqTf1Dq}`4g>Ov9C{%*cPd*c^}(Mq;XYkLXW^v|TK%7iYG zn{}Qp-?HzVHX*LhKSM@7#nO__)cQumeJUQ(lSV0cWJ$Rstj{}x9Hl17AmU72d4VML z>1y#-?uXIGQr18K)sUpfO5#G3CYi#yTAtSseyeE72&cqOi@4&_7fzW+QGiiq(AxdmS19vv9@~Vo|WajJFeX;IT_0XW-qFE{In@qr1Tm5-lPj{zpnWu3C_0c3}! z;C84+P+M_%<<}sF*F}xosQ&;VO0GDjc>yuCb;JWPh?rTyWk!aTheX<4N~Cb%=E>KX zdR6-Iv|2z1LQCSQ0mKT8XWSYoifD^rG(;I)V8o)N(lB4M7Hn9hR>MX$_}SSC?r=Rh z{0ji#6!|g(nxu&}Hd#E{tnJx`yiMJ`M9YIqlIEjoqaB%+2xZ2lmWR2xQmxeCDnTb2 zjmiJ1m(x|wUENx(RxRKD5F!<(Iwz(l+Q9AnDUt{il+=gg2z+oU{*k; zPg#{jz)!rwr`lSP&mlAOE>;zMw2)a%?;lzWwfF;Ob}EQUi@hBj?t5idcwI;C7Z~wj!;c* z`o&k8Clfm&GIDy{GCDhq&Ff8you3+2Ri%)@^!gc6qZEx@;K^kk6c}xy0~Md@X~j{j zvQ@?S*BrPN_;J{lRuxCiI^>AM3ciQmIJO5nn4eX!NrfkPT%+Y#$|ok~B&TGi~S`q&yr9=0~6SaWQ1hzv7t z?Y#=~W`}Af;JIPpGuo1!Dw&B6BPL*GMxhzKHd0t2?3z)CT{Wsvo$E1(rlhQZGX(i2 zA}p^yiyPh&d|1>y{V04Z`dfn3ePsl`&#*_eYkM&r200&)S@!T_u8N6wm)XLUJ z;ikpaErUd8QWDP@(frEHKORO_g%1+Y83Bh~>0(aUFbXN>w7P_AR_-!YZZGDGQu>)XiTeA8t0r!)E=*0A$WrrI1fWS5~uGD8p10O7~~$Q$y(|z=Oj}^ z1EWsLJ+3uvNV637>O)urX#JIADYP2Yqm{7<;po^JAlyw>q0kB>%|a11qMw&Ky{<$btP9E=rci*YQjy@s4;MGp%CI6n!g)YMmhnS#ZhRQR8^S*68He~bC(=%q(& zb*A4vpTo9n)DJ`uq55fS_?Kh!D^7@YD~4{2YjY>jb@0;q!<~D7g;k6Fyf%^INdK7p z?Ao(mEwb0mIULJJ2wUuXPaQ@RJcNavHKq%@x0=XexWTfht;_C}sO+#%v^%`He!?qa z9rUJ_+l4M1S9jC`bW`h`R=A#Jwj4aKP%k(S@}F{^7biT>=-_54kmm0ktsZdA0@1 zy?Jb0TpjP|#a_Q|XB(SST(K*7zrQ%UN%pGfHj!8>X%#ZIT=-l0ynVc${`3Bi&Juop zN9%c8G_uah;@9TKHJD$I`e~ZLqet*M-PbZC}c^&6db2s13 z4q2aly@V<{il@@o^0H%F9~zMG-y{7zi;=6vvp#QdJjoS1G25L+<1i@d3UHf|jfw8` zhQPk~_V)*FYO@7z1<`(ZLYT86!zulu6Nrv|Sra9cO+K=>-bpG7lkHg_nV}OqMD&e> z!e#dnNA7onsUcnKIo(O_G2>A$Y2zR6p=Q1qm%s^>1?8`G`Qv=9Rr z=DSh;w)a-q+8uE&?|)If4LTbf3Q|wfPll+}MAJ?~&NtR1pGehtP&}yVKiVX;c8WEj zfy@fCjmOH*U!=109fbuiqsi@C2Jvj^DsJNex}EN%q0Q|E?^ji#E2$H;%mivTY|!pV z#y#=)%)40Gn4gKC?n0g>N8{iX zRC|=sYk>j1+UPk;6yvo&|; zwgfN=vH^Ze6Y8N38goeb?E)|>yqh)Q`uLQ4;^YJWwaxZuYR;?Y(SBC$@2dOrP9I@N z(vs_L?Kk}l&olPub>;R66psiA=R+YQ$AjUtD0-T}v25!EKXZ7~sa|uJLdv2aDK8rd z%*_8iBm?t7?DU@P?$3xA^m@U%^=Ni#1e|2SG?UO5)`?26niZ*=%!`NtVQa9A+QE+5 zvhEk&F^et3PBhQi-lf>ub94CR{CL*!mEI5zib;t41(-K~)eRj)S|wAuzSsBPU$tqW zs6|{?Om!jGpC6HaZyVG*uiJbyU_xaD1m)k>^;Da++bOreySuvb^E@gbsnjd+d@04#4F^&%LyMVws-2&E~-r zNNLBNonbKyIXIrm3H>HA%q#gEG$U7*r#DI5pdDd`>b(wshVcTo%``sW;xLAteVqQ= zLGQ4JigJeHI*Q-XHCT386Jp{rK}v;(fa)-Ls{@e9VV#*kS2X@NJ#K0|4eL+KhWY6| zfPjoh93Ar`npO21*oI!3kEKeaW#y@OWMtRK^5#B{p?$*L)?k6iR(w;sY%Lu)&&c## ztoATabyIw*e)XZcDNe5wa;VBFV(c|ib)zezSqZO^>LLoJZfrps#y*sHiCeVS4e3WY z><`j>pDV=g8@U1(JK_O-3pZg+S+leSeI{WP4R&$riZu(0Po4~DTyxj<5oGU}ClTqE zMH=2H#1Z_CGW7eQtT#L~pgCb#zE*3Rb#aO_Kgm{Oo8Cv%Ju;p z5JOQmHC+P-K?hugcflk6GJwxIPKOU=A36-r%ONHyKpflzgQ5`%`z&MTFRR4d?)+BwNd~Ag zLRw-r4uO|u;5=A`L>uEDwG1##5`iY+N_uxDV^OO9OFmZz0c{niH-nMju&U;r$r2b0 zJORbQ`hB%6B(Xa=0|qZ8n#!dlWJ7zJ2_7(HkX+KL_g|i7Q?z}jskzVwLxftHa%t~u zUA8KMr7)ac96q6ips5+Z{*Ib~=>5pTb)1hZ71eq-tT#{6KU_|MIlzXwBow!0hkp$L2@w3?6R?Ia zYyip^hoVlpAas`ISIFfz7o8yZDmtc&!1^&}NV}JF#e_^dAhER1A?QRERtr#@?xvo-T+PnT%hf z?%D$IfSUtmbX=mdhQb}=A7sGdNgJ`;p&n{CyaX`N)K{FMcVCZFm3*@bx(=m`CGU7N zMSa3T>`>GsEEN|MrpGQw$4ES|*WZf!&8h_`+n0FrW-vHt#(W6U!BfvS*XI<1hy`)O zpnlA!&dk}V_h%9D;J}sbNEX576kcB7*d=!p*H}fF6W&j`+j#6ZClLx}SM~AZb?9Rr z*DNC_b0v2oEatmao$cUF-7YaL!NkuRD4VmCb&IGleQsYcJO8%ADM37j|H$n8bSX!K zvI=YYGBlaH(aUr@=yW9TP5YjIAM!qcx5?|I;B8l0+ud6Ibc(Z%xVW@goP(Ar81Djb zibG~W&?dvplrM;B_O}*_(WRf-&>F?D3nyF{yPYo4VqLYxQH0)6BDN5md5wO>N0vQ% z3oE2InhGSpu3eYc)LAc@5(7jMU7hE{QuD`orXDS4mR?w!n zdqwJr?lS8)?{5vTyDky6&rdu{5dIjHOTLnnxi>_r+`buLHS_JNf{-OkA|RpgkCT1* zOw;Yhrz~|~!Amsr?dJ!#qMJiZAHN-ayuzvpN6WYZ{@F~61X{n^?pJ$|_ElGxozpix zP|@`t;JaD6cltVCdfrh%HA6R%Oeow)Hk&hBuWU zkQ#C15qT2_eukh#N+RmCy6v(Shb>3}_Iu1g zb%#C;Rtpj&_R()+15#S8QiX|QZVH-Yy|qWwq&^RSz0d3UqU%|36q)rZz=6SsC-hP@ zf=K!ISQ=bGc2s~`A=Am>u-vlJa`6``AU)>od}o_F?Y6Tz+`nv~15FR%Mc+xXd8z8& zs+n?29n=tCb|7M$idb{I?28V8(=dO!A@92sTQ?xihu84nd(MW+@2kFpmnTm?#xI4Y z!Zq7R-)w#nHdz6*B$FTYg@IFhK?{!0#VwFM^vN*p;#{LyMjys4infkJMOyE{)H!>y zyiY*^XCKbW)BpVt?Z7^eF?Vo-U%n^%zaC}h=k-V2mo6RydQ$_^Nq$6((34$g3l8eS zG#LVF9z4hNfj_HU)5YvsqJf4+Ps$$;Qf6SNM$Y64jD^WeNIg{7@BrK;nj_UYL=@BTyPu)c&wOI}evk!8{RiV0eTO#tdM;f)%prx-#x*Cjck| z@f9`)#^$qx$I7mVu>qTz(D4uMVR?BuLYeT9ekUBhlJ2lrg&+m!-(@dggp2cLf(3E! zos|-<0VG(7bF9D6QZNCFtKYs$lg`ml5!^F;*@=qz_O4l}(4WhxYjkrT+3YzSkK#Ep zyb&TI+J?$wLbG&r|0SxMwGoTxf_`b^q-o_TiubwbyhEp|Z3#_lK9m(Hqx&%J%S~!7 zKY(DAWULZEH;;DGX8KVmfgUKSRuLHA4G4TtA@^T*o6m&X&+A}yX+L1iFCam8v<{7Z zFjh1t-G|=$j-r3Iz{f%tMCg5kGB;nPqKu`HE}UR~mSSQy;A7-MvrlgfbtA4J^w> zmOn}-1!;;9oOo&Kw=My6754LMhvggV}^Fe{`In; zIWe*O<{~8cD)?5j1@WczeqkXoX3xMp4J=mohWbt%N53CRR+(OO26dx74g%f|-=0tw z2jiXggY)7!+l!Nr#etNLP;eAg=@TK{A%yLpBj)dP_N(N+7p+H*CmYA_WK=MBF(@@gF#)44vhxvB&0Fg z7Pkiqce$r$TLI4F$=McXzBaaiOk9;#XVFoBmW7NQKIf=?moC__%@ei7z4)V1I(Q$k z$mz2oyvue?VBuxSQb4hB9#qj{hjJ1j!u)~{v}WRWhlL&Ey&Y78 zD&?U@HOX^VY&j(%cp0IY&al*?DCC34m$qHA;&E_H4--BgIqpnQGY%jv2?$+asQm(< z^ix3o)6R?=b)bA$msbSC`%ZpXEw&`+40L`7!qj-UUMT7*PZ@9uEp7lH%cxH-*gk4( z+9}j`eX?;Ec%A|{7P?1(uc}qjTD|^03%iMosSetmG)UPu=b*ZaJ14x5>vmATSA)-$ z_xtD8BnUvw*5GLj7IM!G%?(z!=Hkhw9O;rU1@GW$6p!Z)>gdfnw$F$QWDV zo}HkY1&* zw;+FU7?3!`_&&(#%fXG2NuuQ9no;==@5j=c_75!lUh`;Pu(nTaeNueQ{i$)|$Wc{# zuuAD%(naWsP&`l3jPG)>rtn)(R(?&Am!8D8cLT@QdFK+zd0q_clX(1Rjo1?3FHBKMAY}5QU7txC6cUQ}dz#+wf`QTStj-uJIXznBGTMU-+ z{2MIq^b{w#iAjuzu}Bp%|x(dzNZEL#UMTng^ympXQuCK_g=pF`qphZm(S0~-+w+*ExN4NtEb`)&%R0) z{O|oDlKOL_;@B>;GBoqO;Bq!W%Ikg(`^$HxiR1-L>_M@`E6kOp`bOOJYiVNL;L}#OKRFes!q6QM!^haDF%lYYbmKPL@9lJeaNEMR zGF;I#?Jat>YR|>>CfhBbZ-uKa7C-B@6F%>AA#tM%d`tU1&;)l~$LS8#LaUD0_d&aE zv3*zF_MgSP(DPh2$G= zpXGvTCoSV;bhd*?k{kN?sob;Of}x~VNvj*;c}3ht{IHY2#A^Ey0%L(xsj?4?j-dTB zOj`glGKfT>uOcWdY-K#}H}~qc+WjuWrGR}UZ*EqVa^n5DpaZ0>947fmA|4nPln;lv z `j>M=Jt7HFtOqOWd{*)C1Gk*<=L;!nUA+cTnh>md8BFv_Ql!-7w&u{5|yX;A!d zk)u=(Bp_mTG!d*K(*xauv#uCt`;5hEwm2PvkA@-!%6EU7O=dT?;oA`74sQ3= zefLzK?y$HM%f8C0nOa@WP@eJyY0NW53Q%$D-7_OdD16zGF7D@Aq_H{&3XEk@i^qb( zJREIhxnXMoIKSUN`){xZYZk}f;cNB)w1^DTtmqf^&5!;y<9~bQyN>%~g-%a6xO!7a zE#wQIAHJHO0R0T~w%|ydM#&d9#4Iot+J7jLj+Qh>?Ra~GSd7mP$OYe~2bu6LQT9Er z*ANh;3S@cBGg2i+IC}GQ_rT*6e>%<~J?|d`(4Aq7q0|=fkob53LcW(|aU{P@%l@=g zsbFlw)1M6k=vG`LpG0GbPw69jiGmt1dHh{ZE*P-!!oIqO1WMlUcZrvQ;ENCYp`n4P zSD$%Hi-2vlcDp=w;0aATnkUj94;^gNo5sQbPiQX*(_zDs9$|s5&)K`Twk@|1GjJw# z)2l06XXls>+>hx~_S_x&m;$NR69Ni4$Iq}Q?eIfC3inEQ8iJbPJ3QnV(*;;4` z&BwdyYGhvKDhKs;aCiOmiu&d)*B6b3DBt}ReIcD8f}>HAqlZZD8-;rK3$OsMtB$_% zjafuu;71F=;Qg@oM>Amx4uHN)29R+}L*|UA9nt zHXd7`28eqZ&pnb4GJq7GjdH!-*!l$82ExNaB{sUB<&~?yfp3Af54UA9iFFdjGkNRv z{lv75j#svPFOIbZZbLp_(W96u@>lnC8o#C5*KtSnr+Oo*2S5DM2nrsxp_^iXhVx&u z9r$I=KZHGhR+!wsQ&~}XjX=iPZklt|Q`EC)gsemcgPQbwLn>ZuXm(w8P50_v=QFutH5OCkDm~eV=k&*JNAAnBmLPyn_7AO<^=*iB3IT+*> zaD^Bvtm8o$&5y@T*2_o;=%a_#ka^Dre+rD74_1jM=be%!`)Kynzqg<9{)J_=4JoUX z=X!80rL($nhZmRv)uPBqG8_aPg2`n?!1yXuBMZ;V8c8pi! zRe_69$K8OY1LMQE0)AuGJ~ZO3=?7D#Dn;B|_*NX(a@(Dne@lG0cT?2sGAp>tO;mpW zJtM3V7vJN2bzai}df?k4gA=;&f-uLhhlUuFRt*_T3AOLXHOd~JM6F1$Y} z8ZaggLNNiEjoCDrwPM^QlpSiK_tHw3z?Cjxz@WBF3nhTY4BkARS-~I4@Hwplak(`-12CVp`RMNxvFzR6+QAxlX2`oJ zOa9JVdFylL8IHR>{Fuk7>TB}Jc`k0U12}nni$DVlCydm`?c-(TauF&|a3u-uOx6nc zAe;y!-R5#3+$9mXk1Fhi4++zMwwlTW8K<|Rq~Ja14v~W0O9A1M6f~9je*Cfin%6L1 z9hhf!Mh&zN(we|2z=}TxRVtN4=+Lk+BwwIjZE<`4Y5we(o>I1A#J(Q9>nlYywBGgRvbw^hL_eh(# z#J06!F8Zvpu*dK+QjEghXK+vo(`mYDJ?1&CvS=62hZVjHOtAvPCGT)b3Tb%%eZU{2U-Wq<33kMA`>A0G z`g&r9HS7L2N;8OX)>DmIgt=53ehE+dK6b7oUpGC7eXl;C;YF_UXE;4M9{HJD5{ zyt30T= z`No;}<$^b{8RQchyuaBvvX3TD29qF` z4T7wU?9;{4!g7vt!sVE((iQ>dOQGJr3fzZa!Q!?Oayrim;Eg4+T`oKpz4ot;ol_y_ zi6*BnR|bsis2!gVQGPj6!e|kd4lw2_L^L`Jns62ATqLmN^$l^4+`qQ+$lzJuc4@WC*wz*|T<}3PLPExD`tez-2wa2Z27iqtI3~=Mf$KB^!jLMnHM# zLHqc6r^gZuil#}ae(CUAl}N!>UfOZ;W35hqyrIN>W!e#90`$@NoXHx&toJ6tfoXpP$N#r z3o~+UIhGj%m7yMfv6EOJ9cDq1#CQmlQzZNdlLIRt9pR?lcY{73*dxepM9*?@B>8AZ zS#*IuhrEU$`1}2lbn^&Z67yPz`7x4j2v%!hi)!5M$ITe?lMP%!2TW2*yM{voXu*505 zc2-D-MVsK^FnTfvl+No;Uxo5E_d70Ng>f!k$=X%N+I)ko{bHn>tRJ?uUi!1U?a_`L zRXJK*qhw@mwE~Kb!>>`apeIkhu*v3|6WbGK!yXyCd^IFd>ZCGlbHZJMX^CEjEOVVa zKT9r0zb7#h4fmPV7%0KuN*&qp**rY4!oD*hA)!KH^RQX7LE424+tfGhirQJ)ko zg97C8%C5>Yj5hQ}TJll#QS^=y@=^Pu1?J9@aYwZ#v`qqtDaf0|)xP&|m)}o?#aU$V z)eRO45Z$zdFV*k4&<%wJuFZo*9Db7HSRn54=1A8|lrN2cL8c<`Rp6fhrQ6Xl%n+S= zb@f#{&w3X?7yaNU#pD(DSNC1sK+6(YIANdMIlxgLMStgc@=3Rb0knvwEHWd(Ww;@P zyl5dQMgR!DEmr2QMp6o_e$Ij+mqM*Jz_isuqOABsM9=ub|hFsVx9EybJz&eLER&CS``MQ=pQY6U(2)za(I z0pgA*W4uYbvXSys2w2j=C6^FcF=(nz8T7)GPKtDhHO? z>Slb|HyX+h#!JlM@6(RpO?}==EGr%*M+jMjH%Rsd(^$8R5Ph*EYt>&sM}VI}eiA2G ziQ|Xn)!_w`eRH|??pL46LOaV3g&s4@s!9EdZRFTnK_TCa$j!rZ-6{Zm^4$m?un6b?7uEgc(Q`G0Nc^O@N#J_tVB$L)kY9>eH1Yh1WK8 z%!h6=Rmg{?u?vCh0DII{ z{u1*vi6AQQ>(sPuJIUneSd_POGWA+8Dd#gBXk+*KHs+tIYga^kG#Yc$0rPqXhWw3k zP_~XTyIX9wa9^B3&?i>3c-E{Y54fXiyr=QOqM}A3;%pO6T@{;jRc@F^5{gp z(8-l)*}JVmaDHWfHJ(>a(V_FUuAl&%^1o00E&rE}+}I>5lLN*@yiQ^j4#xYGpq&>o zA&skegsmMNz!w}3euG(|f{V9`(mpv052MS8{eGUPH6#;a!cY^B5mz3X_W0ah3j%Km7l{Y*rHwSY9>X%jpIk z03byM0D$rTjcium%+T58f7i}prilEn1jDyH*74h7N&BzVSWIYTY&e|hy`!Mc58&QR za%Xdo7;PlCxZcd+pdDjUsO9>gwy=Rjm2*=Jzj* zs@pRzSBmXr71XNss_n~HRW;wU-$eWAUAl8-@7+TcQuxK^RP@ZoDtW3^Rq;~(cP_1k zn;?%;n~tKUT)DO&Z#9;$#)|J^z5K4+8ciH5rF<7v(_H|;7Kx=^wa+!|SbBhD+kI1x zHfm_QmSFH#RXmk9jMP&$x&juV-Fu@|VT!CtSwVB~s_UFcGflII3lsCEB@t?amTfYYVXszBWmu=P8OB>;*z77H3 zEc@#)dj4VgGn2E|a6z;v0ic!F00bDnMT%hI%htC_h>Vfgqe;3_ES%qesSgfo1 z3)E08w|j+F+y+`u(l}4Nd6~X-t?263MpQ2!D#tcsQn%CNbZnt`g5NB+0>r2VcM1WI zWHidBesgHgn3?9BUzL)+CFO_h&)Beem)*HBz-G=+DB}pkX`OIkwq3%{Mjte1 z+PVf%<)k7;WbGj|wBE&GnIJw)w=y;ax7AaD1W*kQsDV+L zPkwhU=a$cO(rUnJ+N@UI2`&Ta8f?#qfsiRFUun2-tEOWs2+9oxDj9>Mr9HjaS2QLM z3U0zjm-0#^{<9mQB$DAMZ_Z`#`> zgXV$fvF4)<+6h4VAku><1GP24Ywy0=iVma~;NdqntxpE2K-fIpzOMe8z2$cO7TfLn zh5v9WSE~>mbe-4&Z`RW7>HVCyem#Eu2j9NGJD3x_|2=*~X|*U!M&j9Cfv4{61*QGF zy?;rF;G3RVbFF1|^=V}~fD90G1}$G=a5Zi)LLLD8|Ec6GqvF{5eBHQ1@xJW8oH^|1xW|()T3+?P z(YUhwmM>YYj}wn*(<^?wFZfD9EL5`I!2b#v)9d=f(TADqx$EmWljtB?T5(7&9$2@G z@C|(x64hn*@fR0=$LfQ8HBhuiLc0hgkk`(ZYOB>JNEIFAQ!k4pMyw)E@)Bd1(33^2 zC+LtvBeYlyOD*S_?~vo!Yz%zW9%!I%0ST+w=5i&HMzU{0W;AZQDDyB~0&~-Jk<}Z8Ric ze<*)d&pONI%f89UO`E<)8!iW^_!|nGJc>6~j@CHv$xZ**r!;vMtbQ z&&P(oMSUKcVuzT0XLdw+PV_g;`U8Zg_E(O?SPqvizoQzbb1iB0Q7#0<2E&^#&Bi;(JSyot)ub8W*8lK}uspW7Fj*2Sa|$_*^g5 z6422R+kj8j5f~HRR3UCSb}jd*<$RftwNct74oZWf)qUG-e|Z>%OL!e>JX_~@y=H<16_k5nyqvGEpuF&5?7>?C3Fiqp zlYB5X`Gvku)K*T%a>X+eaL=B`8KeL5fNDs;@L1pdZQC9YT+u-EG-Skr79hTB9fdUu zR6N3gXyk661ei*W6RDe{huHgfaOqpbAy#^0ieVx}d}-s%|B6p&Oz+|rF`YBT-l%t6 z926A*@NraKb2$mXq9q&EOLkA_vJN_d0AkB-C+STwGB`iCTb29TDoV&q0Tp7tm94H{ zi>H8xWt~gjgTOa7l6}#uUkVMQ$x$mHJ_h{MpEYSKh#nLk=-tEmC2>+Njr;=JCjc9i%i`AS!juY~-7gOj#NY8&JKJ7(PaV)}{ zz_C`UcUtIgXoh+v^*7KFggA%ek~0KZfjAT$_lRs^H&$Ek^6g zpweOua-oFy76U-!Lzqq$NtI+moxjnTI?y|BD~jBkm&@6aEaPc%=O^_Y>q(K-P&Wx9 z+$KO>&?iA`LKjkcha70{c}DIy4{6Zol9?-%BU3FozPGwqlf-;1Eamek1YPB51F+YT z_r9AI9EAVDG)_5hlUv>h9$46pEzNtuxJF~@j?VO2={HBzH{MEmz%e))Od9Cnr)jI= z9w=VWaQzm97>TV{HABh`U|`e!kX3!Cdka9lLr=WeF%;TjtU~Up(}EzdTBn((B6O~7 z!v`3bf=qDTryF`QfyY$)5D)C^M*fu)}LVVJF+yo0*CN;Ei3{80- zW4MeS5Q}|10O3i4_?uvMN4a<`KNBYH?pyn)7-x~UL=%Nqvp0mEA=Yv=ph@6DaJzd> zZ@h~6WY1|D-kLxIdyqI6dpl3-cT)n3_bWY-;MiNUAhjw1NgAH0Q>1QSi$tPbz3&Le z{J=93mdMfZz5a5G{nYnsDDxzday_atphN(w?za&hGP41+l_nGl_69N#n>S55q|_`! z+DR9ECV=kYpt)ud?n>?K2NdwLX`cdbvx)}ObuyeLnTz^4{zQ*)$8YNy{b^As$$24J zL7cwT&r|0_UA#gdw9=nGI+3y9wYzMYs@Rlz{t|l7Re(qJ+WE`#a}-m-Q(P&0Y7tL| z7lE;gvUgnr%FmA(>kZF2hN*S6!&%8^d#foWQv_J=GyJ}GfJU**4ozDxnA<}#yd*M9 z_5?jvx}9M#L6HD=6=dq(Z{X=UuVn`q7_-;HjI5tn(v5 zaywJW7>`f95AkEpTHqx1hU)|EZCP$5mGw?1oX_R$xu=RX4~XZCgiheM?Zrc0RCbMZ z$h84oUEt*8B@#G&Ulk$u=9(n}7AWq=h?n@1L_*}4oS#CBdw-i+22GbUGcIkG?-Btn zU=xvGv)(HHYW0U7maI&^SH5XLs3-i$_XTSp zWkO`fE0ujr&@G=niPZ{FIzwOpbT{_`c10)RnxG$2t0gT)QTbFd!jEQ{WEsgsJHT$$ zqH?~00(=j7Az3iu-XPTXPJxC%T|e=6VZ)`Ha-cgv1cf0e3Y@+^>K!H*xIK5|_Lrt=( zygVl)Q#+phQ-JjE#_c2l*4C>?icbvYW(2(A*#@(h-A?m9Mb6RqjhWwDK}Pal$eGJ0 zU2l{qDMhLHe;nwhC^vmk(xIQKAd;slU|_CzFi{!#vKw@lotnl1Y2bR6QnKgHyM88@ z7(>vK;>fg?;#QOuha^N{zZag2K-^V}(!(C&*QWK1j~Re9uAako=F*0MTw|gAJ4^X? zi53+4v1A?49jNz5L(I%^+~vNyV;LyH%(!w`5+ z6fEa>O>I79d*9iNeyP$KR_4lU2T5J4hrN@-e5Jz(n23fWoz z);WWkPfQe*uYpM2M{J~_-f%Bn(yXfR9Qa;V%>jxEj8(!?$1XQS z2J+A06FO*A9gZXml&7ZQleFrfrC(rFE($Gihq3}*LGX-=>baU(vhkgUIp^v&sNeeZ zV4dGF*lI*o9bNgU(O*8zl7X;GaxFO*98Wu&Of(5^xG_l&<08r=qQmr(kyfH;mtH@J9Hq=Hu9}cY1Nv!T)1+j?6jXGiZV~N523g271=2ysMDANlH z@ioFG`7oZ#>5_iOwLdJH*8?=paMd(!=JK$G&WmLaI;l|-`Q}yV-WB1DOkZ7u5Sl2( z^|u$xX+{wl)^+J>Z;da_fI7v=s})39uM~$_dPiYV%+Z5CX|{_1SjhZI+Vue+ zSvo9#1J%?oMSOcNoU0$|Ziqf$mID>}(py$T=9Ibkj^4=k^q(wWr6$JrpCT3c8)JyG ztu&vY;U(SRGMJdrrfn|x$a?p}+82gK(#=l!#3N1VsE$|SIIJdBEW`}o_QFI*pzlQ} zwS;)&1_zwnv**`i%FR)w1BbC$Y(r85x-xwTO^oyHaNGrgWAL(-dv);UJEWTED_0}8 zF?vHVodG8d`Ct5$Za%_6zpZN#WF0W4ERDJiJhXmcCD(3zLRsn3wbh`P?9*{t#S}I~ z>9I_hzP9!AAKzHaCI#cX9cl0kX`v*Gi2Xp^(+w0cPRIxN7yYPz#9S=ak7WL#TeH^Q zm&GmEYM*IprsLPUMH1#h2UVa-F?G?=!eO5Q=iuYL0*pg7S+XD#K9bhefdkK#q}dBG zad8t4m5blwly>7O&(N*??vbsL_UkSbg<&7s-~;!FzwTieZx`dcLtFc z3vea=&N?4INP2k&%GmpHwIgUpv8*T*4W}6>a+n^W1P>=_UN!tmB8D+I(I8 zXNl{p?cMG;O6%jaXI*Elbic^AYr0fo9(>X(SOu9c<1RTX>M-$P%P*TJ;|%VZieLhr zsD^g#pw0K{?Zn~d{5nQRHkv7N>+vly+n(etkVj>GH1Y+%<~e(o`kdj+%8JQ#+DmM1 z2sC?~UAKQw8Q;dCXY?)IWH3m)<*sL`vr(uQmFMjZRQ`qPs7Jx`8(VP5IKlamcKVH* z2Pd=<+y`{I!H%XuS)+elZj0nQuxMgkYd=VXf;9bk#HX&Lj7zXE=qNM^jkCo9+lH0p z$0EZLwY--ydwb08)SV!!*%m52o z_*ykJjfOU-lO%RPje2bT;VeE{$kO2OY-F{;SR4^ac+*j=h14No2ED+Nj+s3Viq}5u zp`B@=k**|oc42LQ=e$5OmHiz8ws&T`bcL$lad=@lQ>j`-RdRd1xb+tjtWgkDavni*%P+I()e`TgG9!}Lqqg!e8x#vTkn->htgN6{k1({s(MT=l!b5lk49i}O=bI(NVm`p+vk?D;5Vh1nLK$Ph| zN=rEbvm+g2L@`Gl8_=O-cw+Yt$!3 zL*8O>Z4sORvLFdf13M~6-x(b#=5ymK$^!%$krpg(AIvl}SN#|DZ|C+dYYDyk#ywl=|0hB!AbGr%+~Bj-&AMT#pQBs{B4$>&*!$d zo{@Yn=WoW=GgOqw$cs|tV#S)gTd`T}UgEoM4c88}92KlE5^Rbp%rE30_KHh63X=8= zw`7RmywIYW+Hq`EEsZjC}y3A|y=Luv|*LC12HIX83L{ ze--skP}_GK_rs37#oQ`=WeWkGI>{A3*b&YvJB9o8c*|~7`>d`_=OEH}a7old9)3j* z(KKgFAAV~A>-`q_Ao+8#?lIbZRoLgM>!2t*@x?r|Os@aW&6jVCSYrE5R4S_?8}b>yDC(4meNwiRfKL8m8x6H}gJi~-cDoyu^X~J+a`}31>iqyYTxy@eG`IeWvupiCCe&o8EPi#8;_W)FHB ziD3R{vD@0YpeqWTwXU(iuGdXZDA$8?4=4(_0E!b|@GXc4_1AXFZ7z+0IK6|&jvLT8 z4NH5+jP9@v1`A87H*m65Od$geZoJw?XvZxw4f#Mf7Ki|&KmOBl%UnN;qW z->llme9Ei#olvN#LiX^OC;O<2y!6tkSq&*be&q{YeWZ6kZ0@p@>Ycg`Jx6FCsS?#^ zCYn1;50Wruq87Xjs-KNbsb#OjeHA3`PcD(2s7IyU_2?h6O=g~jLSc(#TMo`+U%h=$ zwvgcx`a+M@GjAw#s{#(Oz_lZ%_ov$fEDIV4o8Au@OjM!Ye$Fn z+3TeydaxI9vkmC0b6nZiA(pNhujG?^)6gMXqVh%&ckTy9cfkE+ z3t3-Ck>g5Y5hSF+*UR;De8bld9|wD8swkGwt!U<}V0mfOO&xdH#YYz_^RT(R?2o^m z-nyRNw#|f{osn%`X-|_3fzmvg;saX#c+;8kpL$K@IaMzbgd7&I)$q7-WKnzrCv)tJ z`o>i7nz|4bcL&M17pQyYPaIUZ`=~CBg__VJ3z{+Ghp!fHp$W4&9x-TMUl-(n5_th)FlKbHtyLjbBNxP(#hW>elkMOvR7q2`aOv#luoE&m!+AIt+6_I8Jhyu zZJC%;(KGVee7$d`1-hC1ylnX1efRvb;Ake`&=}k8<*HWM``tlto3k-9c?z!sml^eM z#Gb=WM0b4o;UrFJCZN=LN-w!qrpRS35$MO?IFOpjpUa<)2OAy1Anna-ob1L`sRJR0 zR;1f8%E2|>WDaIYBCIyL9K>9>ac6C6DR?#5P+dOSb>zD%lbN<9+cu(n?uG-maJEkU z8vUIVvZp@=O0rYXqqZwMG7;5$`zTp3od`>trPi7RShb%0Jzw&vf&?|v<9PWma(IJ`WaMhg0`IJ;*5k;X(fYM8bd8{ksME&$>^kW?!#f9(G&V0|L~ zB`5wj#J?xGe-g*U{ujjm%5|Txf91OW2K)Dx{!c8U_CI0&v)z9x|5v;JxAG`YzYG8X P^=WlO1ORSy|2q0V9$Eww literal 0 HcmV?d00001 diff --git a/flink-python/lib/cloudpickle-LICENSE.txt b/flink-python/lib/cloudpickle-LICENSE.txt new file mode 100644 index 0000000000..d112c4806a --- /dev/null +++ b/flink-python/lib/cloudpickle-LICENSE.txt @@ -0,0 +1,32 @@ +This module was extracted from the `cloud` package, developed by +PiCloud, Inc. + +Copyright (c) 2015, Cloudpickle contributors. +Copyright (c) 2012, Regents of the University of California. +Copyright (c) 2009 PiCloud, Inc. http://www.picloud.com. +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 of California, Berkeley 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-python/src/main/resources/META-INF/NOTICE b/flink-python/src/main/resources/META-INF/NOTICE index b02bb4d2bb..b2107533d6 100644 --- a/flink-python/src/main/resources/META-INF/NOTICE +++ b/flink-python/src/main/resources/META-INF/NOTICE @@ -53,6 +53,7 @@ See bundled license files for details - com.google.protobuf:protobuf-java:3.7.1 - com.google.protobuf:protobuf-java-util:3.7.1 - com.google.auth:google-auth-library-credentials:0.13.0 +- cloudpickle:1.2.2 This project bundles the following dependencies under the MIT license. (https://opensource.org/licenses/MIT) See bundled license files for details. diff --git a/flink-python/src/main/resources/META-INF/licenses/LICENSE.cloudpickle b/flink-python/src/main/resources/META-INF/licenses/LICENSE.cloudpickle new file mode 100644 index 0000000000..d112c4806a --- /dev/null +++ b/flink-python/src/main/resources/META-INF/licenses/LICENSE.cloudpickle @@ -0,0 +1,32 @@ +This module was extracted from the `cloud` package, developed by +PiCloud, Inc. + +Copyright (c) 2015, Cloudpickle contributors. +Copyright (c) 2012, Regents of the University of California. +Copyright (c) 2009 PiCloud, Inc. http://www.picloud.com. +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 of California, Berkeley 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.cloudpickle b/licenses-binary/LICENSE.cloudpickle new file mode 100644 index 0000000000..d112c4806a --- /dev/null +++ b/licenses-binary/LICENSE.cloudpickle @@ -0,0 +1,32 @@ +This module was extracted from the `cloud` package, developed by +PiCloud, Inc. + +Copyright (c) 2015, Cloudpickle contributors. +Copyright (c) 2012, Regents of the University of California. +Copyright (c) 2009 PiCloud, Inc. http://www.picloud.com. +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 of California, Berkeley 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/LICENSE.cloudpickle b/licenses/LICENSE.cloudpickle new file mode 100644 index 0000000000..d112c4806a --- /dev/null +++ b/licenses/LICENSE.cloudpickle @@ -0,0 +1,32 @@ +This module was extracted from the `cloud` package, developed by +PiCloud, Inc. + +Copyright (c) 2015, Cloudpickle contributors. +Copyright (c) 2012, Regents of the University of California. +Copyright (c) 2009 PiCloud, Inc. http://www.picloud.com. +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 of California, Berkeley 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. -- Gitee From 73a6400666cf2b0f19e1aa85dc48db9b22556cb7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Beno=C3=AEt=20Paris?= Date: Mon, 30 Sep 2019 04:22:31 +0200 Subject: [PATCH 085/268] [hotfix] Fix typo in table planner RuleSets This closes #9795 --- .../table/planner/plan/optimize/program/FlinkBatchProgram.scala | 2 +- .../planner/plan/optimize/program/FlinkStreamProgram.scala | 2 +- .../flink/table/planner/plan/rules/FlinkBatchRuleSets.scala | 2 +- .../flink/table/planner/plan/rules/FlinkStreamRuleSets.scala | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala index 3dfe2ec426..326b026e05 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala @@ -144,7 +144,7 @@ object FlinkBatchProgram { .addProgram(FlinkHepRuleSetProgramBuilder.newBuilder .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION) .setHepMatchOrder(HepMatchOrder.BOTTOM_UP) - .add(FlinkBatchRuleSets.JOIN_REORDER_PERPARE_RULES) + .add(FlinkBatchRuleSets.JOIN_REORDER_PREPARE_RULES) .build(), "merge join into MultiJoin") .addProgram(FlinkHepRuleSetProgramBuilder.newBuilder .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala index 422ec7a6c0..38dfa62cea 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala @@ -140,7 +140,7 @@ object FlinkStreamProgram { .addProgram(FlinkHepRuleSetProgramBuilder.newBuilder .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION) .setHepMatchOrder(HepMatchOrder.BOTTOM_UP) - .add(FlinkStreamRuleSets.JOIN_REORDER_PERPARE_RULES) + .add(FlinkStreamRuleSets.JOIN_REORDER_PREPARE_RULES) .build(), "merge join into MultiJoin") .addProgram(FlinkHepRuleSetProgramBuilder.newBuilder .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala index ab6ca5fb0a..10aaef49ce 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala @@ -221,7 +221,7 @@ object FlinkBatchRuleSets { FILTER_RULES.asScala ).asJava) - val JOIN_REORDER_PERPARE_RULES: RuleSet = RuleSets.ofList( + val JOIN_REORDER_PREPARE_RULES: RuleSet = RuleSets.ofList( // merge join to MultiJoin JoinToMultiJoinRule.INSTANCE, // merge project to MultiJoin diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala index 3e0210d993..6268003949 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala @@ -204,7 +204,7 @@ object FlinkStreamRuleSets { ProjectSetOpTransposeRule.INSTANCE ) - val JOIN_REORDER_PERPARE_RULES: RuleSet = RuleSets.ofList( + val JOIN_REORDER_PREPARE_RULES: RuleSet = RuleSets.ofList( // merge project to MultiJoin ProjectMultiJoinMergeRule.INSTANCE, // merge filter to MultiJoin -- Gitee From 2ac48f0730c42121f5f88daec79209c2fe9cbc2d Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Mon, 30 Sep 2019 09:26:32 +0800 Subject: [PATCH 086/268] [FLINK-14288][legal] Add Py4J NOTICE for source release This closes #9816. --- NOTICE | 1 + 1 file changed, 1 insertion(+) diff --git a/NOTICE b/NOTICE index 30aa331903..ede0dfa17c 100644 --- a/NOTICE +++ b/NOTICE @@ -34,6 +34,7 @@ See bundled license files for details. - d3:3.5.12 - cloudpickle:1.2.2 +- net.sf.py4j:py4j:0.10.8.1 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. -- Gitee From a6188ff593d43c1e9684c1b06be4e655590819c6 Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 17 Sep 2019 19:27:35 +0800 Subject: [PATCH 087/268] [FLINK-14050][yarn] Refactor YarnClusterDescriptor inheritance --- .../yarn/CliFrontendRunWithYarnTest.java | 30 +- .../YARNSessionCapacitySchedulerITCase.java | 2 +- .../flink/yarn/YARNSessionFIFOITCase.java | 4 +- .../NonDeployingYarnClusterDescriptor.java | 28 +- .../yarn/AbstractYarnClusterDescriptor.java | 1652 ----------------- .../flink/yarn/YarnClusterDescriptor.java | 1620 +++++++++++++++- .../flink/yarn/cli/FlinkYarnSessionCli.java | 11 +- .../flink/yarn/AbstractYarnClusterTest.java | 40 +- .../flink/yarn/FlinkYarnSessionCliTest.java | 16 +- .../apache/flink/yarn/YarnFileStageTest.java | 2 +- 10 files changed, 1632 insertions(+), 1773 deletions(-) delete mode 100644 flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendRunWithYarnTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendRunWithYarnTest.java index 97aa2f6bc0..48c415e5d9 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendRunWithYarnTest.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendRunWithYarnTest.java @@ -20,11 +20,9 @@ package org.apache.flink.yarn; import org.apache.flink.client.cli.CliFrontendTestBase; import org.apache.flink.client.cli.CliFrontendTestUtils; -import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.util.FlinkException; import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import org.apache.flink.yarn.util.FakeClusterClient; @@ -32,7 +30,6 @@ import org.apache.flink.yarn.util.NonDeployingYarnClusterDescriptor; import org.apache.commons.cli.CommandLine; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -40,7 +37,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import static junit.framework.TestCase.assertTrue; import static org.apache.flink.client.cli.CliFrontendRunTest.verifyCliFrontend; import static org.apache.flink.yarn.util.YarnTestUtils.getTestJarPath; @@ -93,7 +89,6 @@ public class CliFrontendRunWithYarnTest extends CliFrontendTestBase { } private static class TestingFlinkYarnSessionCli extends FlinkYarnSessionCli { - @SuppressWarnings("unchecked") private final ClusterClient clusterClient; private final String configurationDirectory; @@ -109,10 +104,10 @@ public class CliFrontendRunWithYarnTest extends CliFrontendTestBase { } @Override - public AbstractYarnClusterDescriptor createClusterDescriptor(CommandLine commandLine) + public YarnClusterDescriptor createClusterDescriptor(CommandLine commandLine) throws FlinkException { - AbstractYarnClusterDescriptor parent = super.createClusterDescriptor(commandLine); - return new NonDeployingDetachedYarnClusterDescriptor( + YarnClusterDescriptor parent = super.createClusterDescriptor(commandLine); + return new NonDeployingYarnClusterDescriptor( parent.getFlinkConfiguration(), (YarnConfiguration) parent.getYarnClient().getConfig(), configurationDirectory, @@ -121,23 +116,4 @@ public class CliFrontendRunWithYarnTest extends CliFrontendTestBase { } } - private static class NonDeployingDetachedYarnClusterDescriptor extends NonDeployingYarnClusterDescriptor { - - NonDeployingDetachedYarnClusterDescriptor( - Configuration flinkConfiguration, - YarnConfiguration yarnConfiguration, String configurationDirectory, - YarnClient yarnClient, - ClusterClient clusterClient) { - super(flinkConfiguration, yarnConfiguration, configurationDirectory, yarnClient, - clusterClient); - } - - @Override - public ClusterClient deployJobCluster( - ClusterSpecification clusterSpecification, JobGraph jobGraph, boolean detached) { - assertTrue(detached); - return super.deployJobCluster(clusterSpecification, jobGraph, true); - } - } - } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java index 1cf46c6eb6..6b809bea9c 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java @@ -395,7 +395,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase { public void testNonexistingQueueWARNmessage() throws Exception { runTest(() -> { LOG.info("Starting testNonexistingQueueWARNmessage()"); - addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN); + addTestAppender(YarnClusterDescriptor.class, Level.WARN); try { runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(), diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java index 6acaf5ab66..cfb41b2133 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java @@ -239,7 +239,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase { @Test public void testResourceComputation() throws Exception { runTest(() -> { - addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN); + addTestAppender(YarnClusterDescriptor.class, Level.WARN); LOG.info("Starting testResourceComputation()"); runWithArgs(new String[]{ "-j", flinkUberjar.getAbsolutePath(), @@ -271,7 +271,7 @@ public class YARNSessionFIFOITCase extends YarnTestBase { @Test public void testfullAlloc() throws Exception { runTest(() -> { - addTestAppender(AbstractYarnClusterDescriptor.class, Level.WARN); + addTestAppender(YarnClusterDescriptor.class, Level.WARN); LOG.info("Starting testfullAlloc()"); runWithArgs(new String[]{ "-j", flinkUberjar.getAbsolutePath(), diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/NonDeployingYarnClusterDescriptor.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/NonDeployingYarnClusterDescriptor.java index de0e1d7a95..0264a9f4ed 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/NonDeployingYarnClusterDescriptor.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/NonDeployingYarnClusterDescriptor.java @@ -23,7 +23,7 @@ import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.util.Preconditions; -import org.apache.flink.yarn.AbstractYarnClusterDescriptor; +import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; @@ -31,9 +31,9 @@ import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; /** - * Dummy {@link AbstractYarnClusterDescriptor} without an actual deployment for tests. + * Dummy {@link YarnClusterDescriptor} without an actual deployment for tests. */ -public class NonDeployingYarnClusterDescriptor extends AbstractYarnClusterDescriptor { +public class NonDeployingYarnClusterDescriptor extends YarnClusterDescriptor { private final ClusterClient clusterClient; @@ -45,19 +45,17 @@ public class NonDeployingYarnClusterDescriptor extends AbstractYarnClusterDescri ClusterClient clusterClient) { super(flinkConfiguration, yarnConfiguration, configurationDirectory, yarnClient, true); - //noinspection unchecked this.clusterClient = Preconditions.checkNotNull(clusterClient); } @Override public String getClusterDescription() { - // return parent.getClusterDescription(); return "NonDeployingYarnClusterDescriptor"; } @Override protected ClusterClient createYarnClusterClient( - AbstractYarnClusterDescriptor descriptor, + YarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, ApplicationReport report, @@ -78,21 +76,9 @@ public class NonDeployingYarnClusterDescriptor extends AbstractYarnClusterDescri @Override public ClusterClient deployJobCluster( - ClusterSpecification clusterSpecification, JobGraph jobGraph, boolean detached) { + ClusterSpecification clusterSpecification, + JobGraph jobGraph, + boolean detached) { return clusterClient; } - - @Override - public void killCluster(ApplicationId clusterId) { - } - - @Override - protected String getYarnSessionClusterEntrypoint() { - throw new UnsupportedOperationException("Not needed in test."); - } - - @Override - protected String getYarnJobClusterEntrypoint() { - throw new UnsupportedOperationException("Not needed in test."); - } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java deleted file mode 100644 index a20a902d95..0000000000 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ /dev/null @@ -1,1652 +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.yarn; - -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.client.deployment.ClusterDeploymentException; -import org.apache.flink.client.deployment.ClusterDescriptor; -import org.apache.flink.client.deployment.ClusterRetrieveException; -import org.apache.flink.client.deployment.ClusterSpecification; -import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.configuration.IllegalConfigurationException; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.ResourceManagerOptions; -import org.apache.flink.configuration.RestOptions; -import org.apache.flink.configuration.SecurityOptions; -import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.core.plugin.PluginUtils; -import org.apache.flink.runtime.clusterframework.BootstrapTools; -import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; -import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; -import org.apache.flink.runtime.taskexecutor.TaskManagerServices; -import org.apache.flink.util.FlinkException; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.ShutdownHookUtil; -import org.apache.flink.yarn.configuration.YarnConfigOptions; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsAction; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.ApplicationConstants; -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.hadoop.yarn.api.records.LocalResource; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.NodeState; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.QueueInfo; -import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; -import org.apache.hadoop.yarn.client.api.YarnClient; -import org.apache.hadoop.yarn.client.api.YarnClientApplication; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.util.Records; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nullable; - -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.ObjectOutputStream; -import java.io.PrintStream; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.nio.file.FileVisitResult; -import java.nio.file.Files; -import java.nio.file.SimpleFileVisitor; -import java.nio.file.attribute.BasicFileAttributes; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; - -import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_LIB_DIR; -import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_PLUGINS_DIR; -import static org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever.JOB_GRAPH_FILE_PATH; -import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME; -import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME; -import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.getDynamicProperties; - -/** - * The descriptor with deployment information for deploying a Flink cluster on Yarn. - */ -public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor { - private static final Logger LOG = LoggerFactory.getLogger(AbstractYarnClusterDescriptor.class); - - private final YarnConfiguration yarnConfiguration; - - private final YarnClient yarnClient; - - /** True if the descriptor must not shut down the YarnClient. */ - private final boolean sharedYarnClient; - - private String yarnQueue; - - private String configurationDirectory; - - private Path flinkJarPath; - - private String dynamicPropertiesEncoded; - - /** Lazily initialized list of files to ship. */ - protected List shipFiles = new LinkedList<>(); - - private final Configuration flinkConfiguration; - - private boolean detached; - - private String customName; - - private String zookeeperNamespace; - - private String nodeLabel; - - private String applicationType; - - private YarnConfigOptions.UserJarInclusion userJarInclusion; - - public AbstractYarnClusterDescriptor( - Configuration flinkConfiguration, - YarnConfiguration yarnConfiguration, - String configurationDirectory, - YarnClient yarnClient, - boolean sharedYarnClient) { - - this.yarnConfiguration = Preconditions.checkNotNull(yarnConfiguration); - this.yarnClient = Preconditions.checkNotNull(yarnClient); - this.sharedYarnClient = sharedYarnClient; - - this.flinkConfiguration = Preconditions.checkNotNull(flinkConfiguration); - userJarInclusion = getUserJarInclusionMode(flinkConfiguration); - - this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory); - } - - public YarnClient getYarnClient() { - return yarnClient; - } - - /** - * The class to start the application master with. This class runs the main - * method in case of session cluster. - */ - protected abstract String getYarnSessionClusterEntrypoint(); - - /** - * The class to start the application master with. This class runs the main - * method in case of the job cluster. - */ - protected abstract String getYarnJobClusterEntrypoint(); - - public Configuration getFlinkConfiguration() { - return flinkConfiguration; - } - - public void setQueue(String queue) { - this.yarnQueue = queue; - } - - public void setLocalJarPath(Path localJarPath) { - if (!localJarPath.toString().endsWith("jar")) { - throw new IllegalArgumentException("The passed jar path ('" + localJarPath + "') does not end with the 'jar' extension"); - } - this.flinkJarPath = localJarPath; - } - - /** - * Adds the given files to the list of files to ship. - * - *

Note that any file matching "flink-dist*.jar" will be excluded from the upload by - * {@link #uploadAndRegisterFiles(Collection, FileSystem, Path, ApplicationId, List, Map, StringBuilder)} - * since we upload the Flink uber jar ourselves and do not need to deploy it multiple times. - * - * @param shipFiles files to ship - */ - public void addShipFiles(List shipFiles) { - this.shipFiles.addAll(shipFiles); - } - - public void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded) { - this.dynamicPropertiesEncoded = dynamicPropertiesEncoded; - } - - public String getDynamicPropertiesEncoded() { - return this.dynamicPropertiesEncoded; - } - - private void isReadyForDeployment(ClusterSpecification clusterSpecification) throws YarnDeploymentException { - - if (clusterSpecification.getNumberTaskManagers() <= 0) { - throw new YarnDeploymentException("Taskmanager count must be positive"); - } - if (this.flinkJarPath == null) { - throw new YarnDeploymentException("The Flink jar path is null"); - } - if (this.configurationDirectory == null) { - throw new YarnDeploymentException("Configuration directory not set"); - } - if (this.flinkConfiguration == null) { - throw new YarnDeploymentException("Flink configuration object has not been set"); - } - - // Check if we don't exceed YARN's maximum virtual cores. - // Fetch numYarnMaxVcores from all the RUNNING nodes via yarnClient - final int numYarnMaxVcores; - try { - numYarnMaxVcores = yarnClient.getNodeReports(NodeState.RUNNING) - .stream() - .mapToInt(report -> report.getCapability().getVirtualCores()) - .max() - .orElse(0); - } catch (Exception e) { - throw new YarnDeploymentException("Couldn't get cluster description, please check on the YarnConfiguration", e); - } - - int configuredAmVcores = flinkConfiguration.getInteger(YarnConfigOptions.APP_MASTER_VCORES); - if (configuredAmVcores > numYarnMaxVcores) { - throw new IllegalConfigurationException( - String.format("The number of requested virtual cores for application master %d" + - " exceeds the maximum number of virtual cores %d available in the Yarn Cluster.", - configuredAmVcores, numYarnMaxVcores)); - } - - int configuredVcores = flinkConfiguration.getInteger(YarnConfigOptions.VCORES, clusterSpecification.getSlotsPerTaskManager()); - // don't configure more than the maximum configured number of vcores - if (configuredVcores > numYarnMaxVcores) { - throw new IllegalConfigurationException( - String.format("The number of requested virtual cores per node %d" + - " exceeds the maximum number of virtual cores %d available in the Yarn Cluster." + - " Please note that the number of virtual cores is set to the number of task slots by default" + - " unless configured in the Flink config with '%s.'", - configuredVcores, numYarnMaxVcores, YarnConfigOptions.VCORES.key())); - } - - // check if required Hadoop environment variables are set. If not, warn user - if (System.getenv("HADOOP_CONF_DIR") == null && - System.getenv("YARN_CONF_DIR") == null) { - LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set. " + - "The Flink YARN Client needs one of these to be set to properly load the Hadoop " + - "configuration for accessing YARN."); - } - } - - private static boolean allocateResource(int[] nodeManagers, int toAllocate) { - for (int i = 0; i < nodeManagers.length; i++) { - if (nodeManagers[i] >= toAllocate) { - nodeManagers[i] -= toAllocate; - return true; - } - } - return false; - } - - /** - * @deprecated The cluster descriptor should not know about this option. - */ - @Deprecated - public void setDetachedMode(boolean detachedMode) { - this.detached = detachedMode; - } - - /** - * @deprecated The cluster descriptor should not know about this option. - */ - @Deprecated - public boolean isDetachedMode() { - return detached; - } - - public String getZookeeperNamespace() { - return zookeeperNamespace; - } - - public void setZookeeperNamespace(String zookeeperNamespace) { - this.zookeeperNamespace = zookeeperNamespace; - } - - public String getNodeLabel() { - return nodeLabel; - } - - public void setNodeLabel(String nodeLabel) { - this.nodeLabel = nodeLabel; - } - - // ------------------------------------------------------------- - // Lifecycle management - // ------------------------------------------------------------- - - @Override - public void close() { - if (!sharedYarnClient) { - yarnClient.stop(); - } - } - - // ------------------------------------------------------------- - // ClusterClient overrides - // ------------------------------------------------------------- - - @Override - public ClusterClient retrieve(ApplicationId applicationId) throws ClusterRetrieveException { - - try { - // check if required Hadoop environment variables are set. If not, warn user - if (System.getenv("HADOOP_CONF_DIR") == null && - System.getenv("YARN_CONF_DIR") == null) { - LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set." + - "The Flink YARN Client needs one of these to be set to properly load the Hadoop " + - "configuration for accessing YARN."); - } - - final ApplicationReport report = yarnClient.getApplicationReport(applicationId); - - if (report.getFinalApplicationStatus() != FinalApplicationStatus.UNDEFINED) { - // Flink cluster is not running anymore - LOG.error("The application {} doesn't run anymore. It has previously completed with final status: {}", - applicationId, report.getFinalApplicationStatus()); - throw new RuntimeException("The Yarn application " + applicationId + " doesn't run anymore."); - } - - final String host = report.getHost(); - final int port = report.getRpcPort(); - - LOG.info("Found Web Interface {}:{} of application '{}'.", host, port, applicationId); - - flinkConfiguration.setString(JobManagerOptions.ADDRESS, host); - flinkConfiguration.setInteger(JobManagerOptions.PORT, port); - - flinkConfiguration.setString(RestOptions.ADDRESS, host); - flinkConfiguration.setInteger(RestOptions.PORT, port); - - return createYarnClusterClient( - this, - -1, // we don't know the number of task managers of a started Flink cluster - -1, // we don't know how many slots each task manager has for a started Flink cluster - report, - flinkConfiguration, - false); - } catch (Exception e) { - throw new ClusterRetrieveException("Couldn't retrieve Yarn cluster", e); - } - } - - @Override - public ClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException { - try { - return deployInternal( - clusterSpecification, - "Flink session cluster", - getYarnSessionClusterEntrypoint(), - null, - false); - } catch (Exception e) { - throw new ClusterDeploymentException("Couldn't deploy Yarn session cluster", e); - } - } - - @Override - public void killCluster(ApplicationId applicationId) throws FlinkException { - try { - yarnClient.killApplication(applicationId); - Utils.deleteApplicationFiles(Collections.singletonMap( - YarnConfigKeys.FLINK_YARN_FILES, - getYarnFilesDir(applicationId).toUri().toString())); - } catch (YarnException | IOException e) { - throw new FlinkException("Could not kill the Yarn Flink cluster with id " + applicationId + '.', e); - } - } - - /** - * Method to validate cluster specification before deploy it, it will throw - * an {@link FlinkException} if the {@link ClusterSpecification} is invalid. - * - * @param clusterSpecification cluster specification to check against the configuration of the - * AbstractYarnClusterDescriptor - * @throws FlinkException if the cluster cannot be started with the provided {@link ClusterSpecification} - */ - private void validateClusterSpecification(ClusterSpecification clusterSpecification) throws FlinkException { - try { - final long taskManagerMemorySize = clusterSpecification.getTaskManagerMemoryMB(); - // We do the validation by calling the calculation methods here - // Internally these methods will check whether the cluster can be started with the provided - // ClusterSpecification and the configured memory requirements - final long cutoff = ContaineredTaskManagerParameters.calculateCutoffMB(flinkConfiguration, taskManagerMemorySize); - TaskManagerServices.calculateHeapSizeMB(taskManagerMemorySize - cutoff, flinkConfiguration); - } catch (IllegalArgumentException iae) { - throw new FlinkException("Cannot fulfill the minimum memory requirements with the provided " + - "cluster specification. Please increase the memory of the cluster.", iae); - } - } - - /** - * This method will block until the ApplicationMaster/JobManager have been deployed on YARN. - * - * @param clusterSpecification Initial cluster specification for the Flink cluster to be deployed - * @param applicationName name of the Yarn application to start - * @param yarnClusterEntrypoint Class name of the Yarn cluster entry point. - * @param jobGraph A job graph which is deployed with the Flink cluster, {@code null} if none - * @param detached True if the cluster should be started in detached mode - */ - protected ClusterClient deployInternal( - ClusterSpecification clusterSpecification, - String applicationName, - String yarnClusterEntrypoint, - @Nullable JobGraph jobGraph, - boolean detached) throws Exception { - - // ------------------ Check if configuration is valid -------------------- - validateClusterSpecification(clusterSpecification); - - if (UserGroupInformation.isSecurityEnabled()) { - // note: UGI::hasKerberosCredentials inaccurately reports false - // for logins based on a keytab (fixed in Hadoop 2.6.1, see HADOOP-10786), - // so we check only in ticket cache scenario. - boolean useTicketCache = flinkConfiguration.getBoolean(SecurityOptions.KERBEROS_LOGIN_USETICKETCACHE); - - UserGroupInformation loginUser = UserGroupInformation.getCurrentUser(); - if (loginUser.getAuthenticationMethod() == UserGroupInformation.AuthenticationMethod.KERBEROS - && useTicketCache && !loginUser.hasKerberosCredentials()) { - LOG.error("Hadoop security with Kerberos is enabled but the login user does not have Kerberos credentials"); - throw new RuntimeException("Hadoop security with Kerberos is enabled but the login user " + - "does not have Kerberos credentials"); - } - } - - isReadyForDeployment(clusterSpecification); - - // ------------------ Check if the specified queue exists -------------------- - - checkYarnQueues(yarnClient); - - // ------------------ Add dynamic properties to local flinkConfiguraton ------ - Map dynProperties = getDynamicProperties(dynamicPropertiesEncoded); - for (Map.Entry dynProperty : dynProperties.entrySet()) { - flinkConfiguration.setString(dynProperty.getKey(), dynProperty.getValue()); - } - - // ------------------ Check if the YARN ClusterClient has the requested resources -------------- - - // Create application via yarnClient - final YarnClientApplication yarnApplication = yarnClient.createApplication(); - final GetNewApplicationResponse appResponse = yarnApplication.getNewApplicationResponse(); - - Resource maxRes = appResponse.getMaximumResourceCapability(); - - final ClusterResourceDescription freeClusterMem; - try { - freeClusterMem = getCurrentFreeClusterResources(yarnClient); - } catch (YarnException | IOException e) { - failSessionDuringDeployment(yarnClient, yarnApplication); - throw new YarnDeploymentException("Could not retrieve information about free cluster resources.", e); - } - - final int yarnMinAllocationMB = yarnConfiguration.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0); - - final ClusterSpecification validClusterSpecification; - try { - validClusterSpecification = validateClusterResources( - clusterSpecification, - yarnMinAllocationMB, - maxRes, - freeClusterMem); - } catch (YarnDeploymentException yde) { - failSessionDuringDeployment(yarnClient, yarnApplication); - throw yde; - } - - LOG.info("Cluster specification: {}", validClusterSpecification); - - final ClusterEntrypoint.ExecutionMode executionMode = detached ? - ClusterEntrypoint.ExecutionMode.DETACHED - : ClusterEntrypoint.ExecutionMode.NORMAL; - - flinkConfiguration.setString(ClusterEntrypoint.EXECUTION_MODE, executionMode.toString()); - - ApplicationReport report = startAppMaster( - flinkConfiguration, - applicationName, - yarnClusterEntrypoint, - jobGraph, - yarnClient, - yarnApplication, - validClusterSpecification); - - final String host = report.getHost(); - final int port = report.getRpcPort(); - - flinkConfiguration.setString(JobManagerOptions.ADDRESS, host); - flinkConfiguration.setInteger(JobManagerOptions.PORT, port); - - flinkConfiguration.setString(RestOptions.ADDRESS, host); - flinkConfiguration.setInteger(RestOptions.PORT, port); - - // the Flink cluster is deployed in YARN. Represent cluster - return createYarnClusterClient( - this, - validClusterSpecification.getNumberTaskManagers(), - validClusterSpecification.getSlotsPerTaskManager(), - report, - flinkConfiguration, - true); - } - - protected ClusterSpecification validateClusterResources( - ClusterSpecification clusterSpecification, - int yarnMinAllocationMB, - Resource maximumResourceCapability, - ClusterResourceDescription freeClusterResources) throws YarnDeploymentException { - - int taskManagerCount = clusterSpecification.getNumberTaskManagers(); - int jobManagerMemoryMb = clusterSpecification.getMasterMemoryMB(); - int taskManagerMemoryMb = clusterSpecification.getTaskManagerMemoryMB(); - - if (jobManagerMemoryMb < yarnMinAllocationMB || taskManagerMemoryMb < yarnMinAllocationMB) { - LOG.warn("The JobManager or TaskManager memory is below the smallest possible YARN Container size. " - + "The value of 'yarn.scheduler.minimum-allocation-mb' is '" + yarnMinAllocationMB + "'. Please increase the memory size." + - "YARN will allocate the smaller containers but the scheduler will account for the minimum-allocation-mb, maybe not all instances " + - "you requested will start."); - } - - // set the memory to minAllocationMB to do the next checks correctly - if (jobManagerMemoryMb < yarnMinAllocationMB) { - jobManagerMemoryMb = yarnMinAllocationMB; - } - if (taskManagerMemoryMb < yarnMinAllocationMB) { - taskManagerMemoryMb = yarnMinAllocationMB; - } - - final String note = "Please check the 'yarn.scheduler.maximum-allocation-mb' and the 'yarn.nodemanager.resource.memory-mb' configuration values\n"; - if (jobManagerMemoryMb > maximumResourceCapability.getMemory()) { - throw new YarnDeploymentException("The cluster does not have the requested resources for the JobManager available!\n" - + "Maximum Memory: " + maximumResourceCapability.getMemory() + "MB Requested: " + jobManagerMemoryMb + "MB. " + note); - } - - if (taskManagerMemoryMb > maximumResourceCapability.getMemory()) { - throw new YarnDeploymentException("The cluster does not have the requested resources for the TaskManagers available!\n" - + "Maximum Memory: " + maximumResourceCapability.getMemory() + " Requested: " + taskManagerMemoryMb + "MB. " + note); - } - - final String noteRsc = "\nThe Flink YARN client will try to allocate the YARN session, but maybe not all TaskManagers are " + - "connecting from the beginning because the resources are currently not available in the cluster. " + - "The allocation might take more time than usual because the Flink YARN client needs to wait until " + - "the resources become available."; - int totalMemoryRequired = jobManagerMemoryMb + taskManagerMemoryMb * taskManagerCount; - - if (freeClusterResources.totalFreeMemory < totalMemoryRequired) { - LOG.warn("This YARN session requires " + totalMemoryRequired + "MB of memory in the cluster. " - + "There are currently only " + freeClusterResources.totalFreeMemory + "MB available." + noteRsc); - - } - if (taskManagerMemoryMb > freeClusterResources.containerLimit) { - LOG.warn("The requested amount of memory for the TaskManagers (" + taskManagerMemoryMb + "MB) is more than " - + "the largest possible YARN container: " + freeClusterResources.containerLimit + noteRsc); - } - if (jobManagerMemoryMb > freeClusterResources.containerLimit) { - LOG.warn("The requested amount of memory for the JobManager (" + jobManagerMemoryMb + "MB) is more than " - + "the largest possible YARN container: " + freeClusterResources.containerLimit + noteRsc); - } - - // ----------------- check if the requested containers fit into the cluster. - - int[] nmFree = Arrays.copyOf(freeClusterResources.nodeManagersFree, freeClusterResources.nodeManagersFree.length); - // first, allocate the jobManager somewhere. - if (!allocateResource(nmFree, jobManagerMemoryMb)) { - LOG.warn("Unable to find a NodeManager that can fit the JobManager/Application master. " + - "The JobManager requires " + jobManagerMemoryMb + "MB. NodeManagers available: " + - Arrays.toString(freeClusterResources.nodeManagersFree) + noteRsc); - } - // allocate TaskManagers - for (int i = 0; i < taskManagerCount; i++) { - if (!allocateResource(nmFree, taskManagerMemoryMb)) { - LOG.warn("There is not enough memory available in the YARN cluster. " + - "The TaskManager(s) require " + taskManagerMemoryMb + "MB each. " + - "NodeManagers available: " + Arrays.toString(freeClusterResources.nodeManagersFree) + "\n" + - "After allocating the JobManager (" + jobManagerMemoryMb + "MB) and (" + i + "/" + taskManagerCount + ") TaskManagers, " + - "the following NodeManagers are available: " + Arrays.toString(nmFree) + noteRsc); - } - } - - return new ClusterSpecification.ClusterSpecificationBuilder() - .setMasterMemoryMB(jobManagerMemoryMb) - .setTaskManagerMemoryMB(taskManagerMemoryMb) - .setNumberTaskManagers(clusterSpecification.getNumberTaskManagers()) - .setSlotsPerTaskManager(clusterSpecification.getSlotsPerTaskManager()) - .createClusterSpecification(); - - } - - private void checkYarnQueues(YarnClient yarnClient) { - try { - List queues = yarnClient.getAllQueues(); - if (queues.size() > 0 && this.yarnQueue != null) { // check only if there are queues configured in yarn and for this session. - boolean queueFound = false; - for (QueueInfo queue : queues) { - if (queue.getQueueName().equals(this.yarnQueue)) { - queueFound = true; - break; - } - } - if (!queueFound) { - String queueNames = ""; - for (QueueInfo queue : queues) { - queueNames += queue.getQueueName() + ", "; - } - LOG.warn("The specified queue '" + this.yarnQueue + "' does not exist. " + - "Available queues: " + queueNames); - } - } else { - LOG.debug("The YARN cluster does not have any queues configured"); - } - } catch (Throwable e) { - LOG.warn("Error while getting queue information from YARN: " + e.getMessage()); - if (LOG.isDebugEnabled()) { - LOG.debug("Error details", e); - } - } - } - - public ApplicationReport startAppMaster( - Configuration configuration, - String applicationName, - String yarnClusterEntrypoint, - JobGraph jobGraph, - YarnClient yarnClient, - YarnClientApplication yarnApplication, - ClusterSpecification clusterSpecification) throws Exception { - - // ------------------ Initialize the file systems ------------------------- - - org.apache.flink.core.fs.FileSystem.initialize( - configuration, - PluginUtils.createPluginManagerFromRootFolder(configuration)); - - // initialize file system - // Copy the application master jar to the filesystem - // Create a local resource to point to the destination jar path - final FileSystem fs = FileSystem.get(yarnConfiguration); - final Path homeDir = fs.getHomeDirectory(); - - // hard coded check for the GoogleHDFS client because its not overriding the getScheme() method. - if (!fs.getClass().getSimpleName().equals("GoogleHadoopFileSystem") && - fs.getScheme().startsWith("file")) { - LOG.warn("The file system scheme is '" + fs.getScheme() + "'. This indicates that the " - + "specified Hadoop configuration path is wrong and the system is using the default Hadoop configuration values." - + "The Flink YARN client needs to store its files in a distributed file system"); - } - - ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext(); - Set systemShipFiles = new HashSet<>(shipFiles.size()); - for (File file : shipFiles) { - systemShipFiles.add(file.getAbsoluteFile()); - } - - //check if there is a logback or log4j file - File logbackFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOGBACK_NAME); - final boolean hasLogback = logbackFile.exists(); - if (hasLogback) { - systemShipFiles.add(logbackFile); - } - - File log4jFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOG4J_NAME); - final boolean hasLog4j = log4jFile.exists(); - if (hasLog4j) { - systemShipFiles.add(log4jFile); - if (hasLogback) { - // this means there is already a logback configuration file --> fail - LOG.warn("The configuration directory ('" + configurationDirectory + "') contains both LOG4J and " + - "Logback configuration files. Please delete or rename one of them."); - } - } - - addEnvironmentFoldersToShipFiles(systemShipFiles); - - // Set-up ApplicationSubmissionContext for the application - - final ApplicationId appId = appContext.getApplicationId(); - - // ------------------ Add Zookeeper namespace to local flinkConfiguraton ------ - String zkNamespace = getZookeeperNamespace(); - // no user specified cli argument for namespace? - if (zkNamespace == null || zkNamespace.isEmpty()) { - // namespace defined in config? else use applicationId as default. - zkNamespace = configuration.getString(HighAvailabilityOptions.HA_CLUSTER_ID, String.valueOf(appId)); - setZookeeperNamespace(zkNamespace); - } - - configuration.setString(HighAvailabilityOptions.HA_CLUSTER_ID, zkNamespace); - - if (HighAvailabilityMode.isHighAvailabilityModeActivated(configuration)) { - // activate re-execution of failed applications - appContext.setMaxAppAttempts( - configuration.getInteger( - YarnConfigOptions.APPLICATION_ATTEMPTS.key(), - YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)); - - activateHighAvailabilitySupport(appContext); - } else { - // set number of application retries to 1 in the default case - appContext.setMaxAppAttempts( - configuration.getInteger( - YarnConfigOptions.APPLICATION_ATTEMPTS.key(), - 1)); - } - - final Set userJarFiles = (jobGraph == null) - // not per-job submission - ? Collections.emptySet() - // add user code jars from the provided JobGraph - : jobGraph.getUserJars().stream().map(f -> f.toUri()).map(File::new).collect(Collectors.toSet()); - - // local resource map for Yarn - final Map localResources = new HashMap<>(2 + systemShipFiles.size() + userJarFiles.size()); - // list of remote paths (after upload) - final List paths = new ArrayList<>(2 + systemShipFiles.size() + userJarFiles.size()); - // ship list that enables reuse of resources for task manager containers - StringBuilder envShipFileList = new StringBuilder(); - - // upload and register ship files - List systemClassPaths = uploadAndRegisterFiles( - systemShipFiles, - fs, - homeDir, - appId, - paths, - localResources, - envShipFileList); - - final List userClassPaths = uploadAndRegisterFiles( - userJarFiles, - fs, - homeDir, - appId, - paths, - localResources, - envShipFileList); - - if (userJarInclusion == YarnConfigOptions.UserJarInclusion.ORDER) { - systemClassPaths.addAll(userClassPaths); - } - - // normalize classpath by sorting - Collections.sort(systemClassPaths); - Collections.sort(userClassPaths); - - // classpath assembler - StringBuilder classPathBuilder = new StringBuilder(); - if (userJarInclusion == YarnConfigOptions.UserJarInclusion.FIRST) { - for (String userClassPath : userClassPaths) { - classPathBuilder.append(userClassPath).append(File.pathSeparator); - } - } - for (String classPath : systemClassPaths) { - classPathBuilder.append(classPath).append(File.pathSeparator); - } - - // Setup jar for ApplicationMaster - Path remotePathJar = setupSingleLocalResource( - "flink.jar", - fs, - appId, - flinkJarPath, - localResources, - homeDir, - ""); - - // set the right configuration values for the TaskManager - configuration.setInteger( - TaskManagerOptions.NUM_TASK_SLOTS, - clusterSpecification.getSlotsPerTaskManager()); - - configuration.setString( - TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, - clusterSpecification.getTaskManagerMemoryMB() + "m"); - - // Upload the flink configuration - // write out configuration file - File tmpConfigurationFile = File.createTempFile(appId + "-flink-conf.yaml", null); - tmpConfigurationFile.deleteOnExit(); - BootstrapTools.writeConfiguration(configuration, tmpConfigurationFile); - - Path remotePathConf = setupSingleLocalResource( - "flink-conf.yaml", - fs, - appId, - new Path(tmpConfigurationFile.getAbsolutePath()), - localResources, - homeDir, - ""); - - paths.add(remotePathJar); - classPathBuilder.append("flink.jar").append(File.pathSeparator); - paths.add(remotePathConf); - classPathBuilder.append("flink-conf.yaml").append(File.pathSeparator); - - if (userJarInclusion == YarnConfigOptions.UserJarInclusion.LAST) { - for (String userClassPath : userClassPaths) { - classPathBuilder.append(userClassPath).append(File.pathSeparator); - } - } - - // write job graph to tmp file and add it to local resource - // TODO: server use user main method to generate job graph - if (jobGraph != null) { - try { - File fp = File.createTempFile(appId.toString(), null); - fp.deleteOnExit(); - try (FileOutputStream output = new FileOutputStream(fp); - ObjectOutputStream obOutput = new ObjectOutputStream(output);){ - obOutput.writeObject(jobGraph); - } - - final String jobGraphFilename = "job.graph"; - flinkConfiguration.setString(JOB_GRAPH_FILE_PATH, jobGraphFilename); - - Path pathFromYarnURL = setupSingleLocalResource( - jobGraphFilename, - fs, - appId, - new Path(fp.toURI()), - localResources, - homeDir, - ""); - paths.add(pathFromYarnURL); - classPathBuilder.append(jobGraphFilename).append(File.pathSeparator); - } catch (Exception e) { - LOG.warn("Add job graph to local resource fail"); - throw e; - } - } - - final Path yarnFilesDir = getYarnFilesDir(appId); - FsPermission permission = new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE); - fs.setPermission(yarnFilesDir, permission); // set permission for path. - - //To support Yarn Secure Integration Test Scenario - //In Integration test setup, the Yarn containers created by YarnMiniCluster does not have the Yarn site XML - //and KRB5 configuration files. We are adding these files as container local resources for the container - //applications (JM/TMs) to have proper secure cluster setup - Path remoteKrb5Path = null; - Path remoteYarnSiteXmlPath = null; - boolean hasKrb5 = false; - if (System.getenv("IN_TESTS") != null) { - File f = new File(System.getenv("YARN_CONF_DIR"), Utils.YARN_SITE_FILE_NAME); - LOG.info("Adding Yarn configuration {} to the AM container local resource bucket", f.getAbsolutePath()); - Path yarnSitePath = new Path(f.getAbsolutePath()); - remoteYarnSiteXmlPath = setupSingleLocalResource( - Utils.YARN_SITE_FILE_NAME, - fs, - appId, - yarnSitePath, - localResources, - homeDir, - ""); - - String krb5Config = System.getProperty("java.security.krb5.conf"); - if (krb5Config != null && krb5Config.length() != 0) { - File krb5 = new File(krb5Config); - LOG.info("Adding KRB5 configuration {} to the AM container local resource bucket", krb5.getAbsolutePath()); - Path krb5ConfPath = new Path(krb5.getAbsolutePath()); - remoteKrb5Path = setupSingleLocalResource( - Utils.KRB5_FILE_NAME, - fs, - appId, - krb5ConfPath, - localResources, - homeDir, - ""); - hasKrb5 = true; - } - } - - // setup security tokens - Path remotePathKeytab = null; - String keytab = configuration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB); - if (keytab != null) { - LOG.info("Adding keytab {} to the AM container local resource bucket", keytab); - remotePathKeytab = setupSingleLocalResource( - Utils.KEYTAB_FILE_NAME, - fs, - appId, - new Path(keytab), - localResources, - homeDir, - ""); - } - - final ContainerLaunchContext amContainer = setupApplicationMasterContainer( - yarnClusterEntrypoint, - hasLogback, - hasLog4j, - hasKrb5, - clusterSpecification.getMasterMemoryMB()); - - if (UserGroupInformation.isSecurityEnabled()) { - // set HDFS delegation tokens when security is enabled - LOG.info("Adding delegation token to the AM container.."); - Utils.setTokensFor(amContainer, paths, yarnConfiguration); - } - - amContainer.setLocalResources(localResources); - fs.close(); - - // Setup CLASSPATH and environment variables for ApplicationMaster - final Map appMasterEnv = new HashMap<>(); - // set user specified app master environment variables - appMasterEnv.putAll(Utils.getEnvironmentVariables(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, configuration)); - // set Flink app class path - appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, classPathBuilder.toString()); - - // set Flink on YARN internal configuration values - appMasterEnv.put(YarnConfigKeys.ENV_TM_COUNT, String.valueOf(clusterSpecification.getNumberTaskManagers())); - appMasterEnv.put(YarnConfigKeys.ENV_TM_MEMORY, String.valueOf(clusterSpecification.getTaskManagerMemoryMB())); - appMasterEnv.put(YarnConfigKeys.FLINK_JAR_PATH, remotePathJar.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, homeDir.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, envShipFileList.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_SLOTS, String.valueOf(clusterSpecification.getSlotsPerTaskManager())); - appMasterEnv.put(YarnConfigKeys.ENV_DETACHED, String.valueOf(detached)); - appMasterEnv.put(YarnConfigKeys.ENV_ZOOKEEPER_NAMESPACE, getZookeeperNamespace()); - appMasterEnv.put(YarnConfigKeys.FLINK_YARN_FILES, yarnFilesDir.toUri().toString()); - - // https://github.com/apache/hadoop/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md#identity-on-an-insecure-cluster-hadoop_user_name - appMasterEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, UserGroupInformation.getCurrentUser().getUserName()); - - if (remotePathKeytab != null) { - appMasterEnv.put(YarnConfigKeys.KEYTAB_PATH, remotePathKeytab.toString()); - String principal = configuration.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL); - appMasterEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, principal); - } - - //To support Yarn Secure Integration Test Scenario - if (remoteYarnSiteXmlPath != null) { - appMasterEnv.put(YarnConfigKeys.ENV_YARN_SITE_XML_PATH, remoteYarnSiteXmlPath.toString()); - } - if (remoteKrb5Path != null) { - appMasterEnv.put(YarnConfigKeys.ENV_KRB5_PATH, remoteKrb5Path.toString()); - } - - if (dynamicPropertiesEncoded != null) { - appMasterEnv.put(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES, dynamicPropertiesEncoded); - } - - // set classpath from YARN configuration - Utils.setupYarnClassPath(yarnConfiguration, appMasterEnv); - - amContainer.setEnvironment(appMasterEnv); - - // Set up resource type requirements for ApplicationMaster - Resource capability = Records.newRecord(Resource.class); - capability.setMemory(clusterSpecification.getMasterMemoryMB()); - capability.setVirtualCores(flinkConfiguration.getInteger(YarnConfigOptions.APP_MASTER_VCORES)); - - final String customApplicationName = customName != null ? customName : applicationName; - - appContext.setApplicationName(customApplicationName); - appContext.setApplicationType(applicationType != null ? applicationType : "Apache Flink"); - appContext.setAMContainerSpec(amContainer); - appContext.setResource(capability); - - // Set priority for application - int priorityNum = flinkConfiguration.getInteger(YarnConfigOptions.APPLICATION_PRIORITY); - if (priorityNum >= 0) { - Priority priority = Priority.newInstance(priorityNum); - appContext.setPriority(priority); - } - - if (yarnQueue != null) { - appContext.setQueue(yarnQueue); - } - - setApplicationNodeLabel(appContext); - - setApplicationTags(appContext); - - // add a hook to clean up in case deployment fails - Thread deploymentFailureHook = new DeploymentFailureHook(yarnClient, yarnApplication, yarnFilesDir); - Runtime.getRuntime().addShutdownHook(deploymentFailureHook); - LOG.info("Submitting application master " + appId); - yarnClient.submitApplication(appContext); - - LOG.info("Waiting for the cluster to be allocated"); - final long startTime = System.currentTimeMillis(); - ApplicationReport report; - YarnApplicationState lastAppState = YarnApplicationState.NEW; - loop: while (true) { - try { - report = yarnClient.getApplicationReport(appId); - } catch (IOException e) { - throw new YarnDeploymentException("Failed to deploy the cluster.", e); - } - YarnApplicationState appState = report.getYarnApplicationState(); - LOG.debug("Application State: {}", appState); - switch(appState) { - case FAILED: - case FINISHED: - case KILLED: - throw new YarnDeploymentException("The YARN application unexpectedly switched to state " - + appState + " during deployment. \n" + - "Diagnostics from YARN: " + report.getDiagnostics() + "\n" + - "If log aggregation is enabled on your cluster, use this command to further investigate the issue:\n" + - "yarn logs -applicationId " + appId); - //break .. - case RUNNING: - LOG.info("YARN application has been deployed successfully."); - break loop; - default: - if (appState != lastAppState) { - LOG.info("Deploying cluster, current state " + appState); - } - if (System.currentTimeMillis() - startTime > 60000) { - LOG.info("Deployment took more than 60 seconds. Please check if the requested resources are available in the YARN cluster"); - } - - } - lastAppState = appState; - Thread.sleep(250); - } - // print the application id for user to cancel themselves. - if (isDetachedMode()) { - LOG.info("The Flink YARN client has been started in detached mode. In order to stop " + - "Flink on YARN, use the following command or a YARN web interface to stop " + - "it:\nyarn application -kill " + appId + "\nPlease also note that the " + - "temporary files of the YARN session in the home directory will not be removed."); - } - // since deployment was successful, remove the hook - ShutdownHookUtil.removeShutdownHook(deploymentFailureHook, getClass().getSimpleName(), LOG); - return report; - } - - /** - * Returns the Path where the YARN application files should be uploaded to. - * - * @param appId YARN application id - */ - private Path getYarnFilesDir(final ApplicationId appId) throws IOException { - final FileSystem fileSystem = FileSystem.get(yarnConfiguration); - final Path homeDir = fileSystem.getHomeDirectory(); - return new Path(homeDir, ".flink/" + appId + '/'); - } - - /** - * Uploads and registers a single resource and adds it to localResources. - * - * @param key - * the key to add the resource under - * @param fs - * the remote file system to upload to - * @param appId - * application ID - * @param localSrcPath - * local path to the file - * @param localResources - * map of resources - * - * @return the remote path to the uploaded resource - */ - private static Path setupSingleLocalResource( - String key, - FileSystem fs, - ApplicationId appId, - Path localSrcPath, - Map localResources, - Path targetHomeDir, - String relativeTargetPath) throws IOException { - Tuple2 resource = Utils.setupLocalResource( - fs, - appId.toString(), - localSrcPath, - targetHomeDir, - relativeTargetPath); - - localResources.put(key, resource.f1); - - return resource.f0; - } - - /** - * Match file name for "flink-dist*.jar" pattern. - * - * @param fileName file name to check - * @return true if file is a dist jar - */ - private static boolean isDistJar(String fileName) { - return fileName.startsWith("flink-dist") && fileName.endsWith("jar"); - } - - /** - * Recursively uploads (and registers) any (user and system) files in shipFiles except - * for files matching "flink-dist*.jar" which should be uploaded separately. - * - * @param shipFiles - * files to upload - * @param fs - * file system to upload to - * @param targetHomeDir - * remote home directory to upload to - * @param appId - * application ID - * @param remotePaths - * paths of the remote resources (uploaded resources will be added) - * @param localResources - * map of resources (uploaded resources will be added) - * @param envShipFileList - * list of shipped files in a format understood by {@link Utils#createTaskExecutorContext} - * - * @return list of class paths with the the proper resource keys from the registration - */ - static List uploadAndRegisterFiles( - Collection shipFiles, - FileSystem fs, - Path targetHomeDir, - ApplicationId appId, - List remotePaths, - Map localResources, - StringBuilder envShipFileList) throws IOException { - final List localPaths = new ArrayList<>(); - final List relativePaths = new ArrayList<>(); - for (File shipFile : shipFiles) { - if (shipFile.isDirectory()) { - // add directories to the classpath - final java.nio.file.Path shipPath = shipFile.toPath(); - final java.nio.file.Path parentPath = shipPath.getParent(); - Files.walkFileTree(shipPath, new SimpleFileVisitor() { - @Override - public FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes attrs) { - localPaths.add(new Path(file.toUri())); - relativePaths.add(new Path(parentPath.relativize(file).toString())); - return FileVisitResult.CONTINUE; - } - }); - } else { - localPaths.add(new Path(shipFile.toURI())); - relativePaths.add(new Path(shipFile.getName())); - } - } - - final Set archives = new HashSet<>(); - final Set resources = new HashSet<>(); - for (int i = 0; i < localPaths.size(); i++) { - final Path localPath = localPaths.get(i); - final Path relativePath = relativePaths.get(i); - if (!isDistJar(relativePath.getName())) { - final String key = relativePath.toString(); - final Path remotePath = setupSingleLocalResource( - key, - fs, - appId, - localPath, - localResources, - targetHomeDir, - relativePath.getParent().toString()); - remotePaths.add(remotePath); - envShipFileList.append(key).append("=").append(remotePath).append(","); - // add files to the classpath - if (key.endsWith("jar")) { - archives.add(relativePath.toString()); - } else { - resources.add(relativePath.getParent().toString()); - } - } - } - - // construct classpath, we always want resource directories to go first, we also sort - // both resources and archives in order to make classpath deterministic - final ArrayList classPaths = new ArrayList<>(); - resources.stream().sorted().forEach(classPaths::add); - archives.stream().sorted().forEach(classPaths::add); - return classPaths; - } - - /** - * Kills YARN application and stops YARN client. - * - *

Use this method to kill the App before it has been properly deployed - */ - private void failSessionDuringDeployment(YarnClient yarnClient, YarnClientApplication yarnApplication) { - LOG.info("Killing YARN application"); - - try { - yarnClient.killApplication(yarnApplication.getNewApplicationResponse().getApplicationId()); - } catch (Exception e) { - // we only log a debug message here because the "killApplication" call is a best-effort - // call (we don't know if the application has been deployed when the error occured). - LOG.debug("Error while killing YARN application", e); - } - yarnClient.stop(); - } - - private static class ClusterResourceDescription { - public final int totalFreeMemory; - public final int containerLimit; - public final int[] nodeManagersFree; - - public ClusterResourceDescription(int totalFreeMemory, int containerLimit, int[] nodeManagersFree) { - this.totalFreeMemory = totalFreeMemory; - this.containerLimit = containerLimit; - this.nodeManagersFree = nodeManagersFree; - } - } - - private ClusterResourceDescription getCurrentFreeClusterResources(YarnClient yarnClient) throws YarnException, IOException { - List nodes = yarnClient.getNodeReports(NodeState.RUNNING); - - int totalFreeMemory = 0; - int containerLimit = 0; - int[] nodeManagersFree = new int[nodes.size()]; - - for (int i = 0; i < nodes.size(); i++) { - NodeReport rep = nodes.get(i); - int free = rep.getCapability().getMemory() - (rep.getUsed() != null ? rep.getUsed().getMemory() : 0); - nodeManagersFree[i] = free; - totalFreeMemory += free; - if (free > containerLimit) { - containerLimit = free; - } - } - return new ClusterResourceDescription(totalFreeMemory, containerLimit, nodeManagersFree); - } - - @Override - public String getClusterDescription() { - - try { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - PrintStream ps = new PrintStream(baos); - - YarnClusterMetrics metrics = yarnClient.getYarnClusterMetrics(); - - ps.append("NodeManagers in the ClusterClient " + metrics.getNumNodeManagers()); - List nodes = yarnClient.getNodeReports(NodeState.RUNNING); - final String format = "|%-16s |%-16s %n"; - ps.printf("|Property |Value %n"); - ps.println("+---------------------------------------+"); - int totalMemory = 0; - int totalCores = 0; - for (NodeReport rep : nodes) { - final Resource res = rep.getCapability(); - totalMemory += res.getMemory(); - totalCores += res.getVirtualCores(); - ps.format(format, "NodeID", rep.getNodeId()); - ps.format(format, "Memory", res.getMemory() + " MB"); - ps.format(format, "vCores", res.getVirtualCores()); - ps.format(format, "HealthReport", rep.getHealthReport()); - ps.format(format, "Containers", rep.getNumContainers()); - ps.println("+---------------------------------------+"); - } - ps.println("Summary: totalMemory " + totalMemory + " totalCores " + totalCores); - List qInfo = yarnClient.getAllQueues(); - for (QueueInfo q : qInfo) { - ps.println("Queue: " + q.getQueueName() + ", Current Capacity: " + q.getCurrentCapacity() + " Max Capacity: " + - q.getMaximumCapacity() + " Applications: " + q.getApplications().size()); - } - return baos.toString(); - } catch (Exception e) { - throw new RuntimeException("Couldn't get cluster description", e); - } - } - - public void setName(String name) { - this.customName = Preconditions.checkNotNull(name, "The customized name must not be null"); - } - - public void setApplicationType(String type) { - this.applicationType = Preconditions.checkNotNull(type, "The customized application type must not be null"); - } - - private void activateHighAvailabilitySupport(ApplicationSubmissionContext appContext) throws - InvocationTargetException, IllegalAccessException { - - ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); - - reflector.setKeepContainersAcrossApplicationAttempts(appContext, true); - - reflector.setAttemptFailuresValidityInterval( - appContext, - flinkConfiguration.getLong(YarnConfigOptions.APPLICATION_ATTEMPT_FAILURE_VALIDITY_INTERVAL)); - } - - private void setApplicationTags(final ApplicationSubmissionContext appContext) throws InvocationTargetException, - IllegalAccessException { - - final ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); - final String tagsString = flinkConfiguration.getString(YarnConfigOptions.APPLICATION_TAGS); - - final Set applicationTags = new HashSet<>(); - - // Trim whitespace and cull empty tags - for (final String tag : tagsString.split(",")) { - final String trimmedTag = tag.trim(); - if (!trimmedTag.isEmpty()) { - applicationTags.add(trimmedTag); - } - } - - reflector.setApplicationTags(appContext, applicationTags); - } - - private void setApplicationNodeLabel(final ApplicationSubmissionContext appContext) throws InvocationTargetException, - IllegalAccessException { - - if (nodeLabel != null) { - final ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); - reflector.setApplicationNodeLabel(appContext, nodeLabel); - } - } - - /** - * Singleton object which uses reflection to determine whether the {@link ApplicationSubmissionContext} - * supports various methods which, depending on the Hadoop version, may or may not be supported. - * - *

If an unsupported method is invoked, nothing happens. - * - *

Currently three methods are proxied: - * - setApplicationTags (>= 2.4.0) - * - setAttemptFailuresValidityInterval (>= 2.6.0) - * - setKeepContainersAcrossApplicationAttempts (>= 2.4.0) - * - setNodeLabelExpression (>= 2.6.0) - */ - private static class ApplicationSubmissionContextReflector { - private static final Logger LOG = LoggerFactory.getLogger(ApplicationSubmissionContextReflector.class); - - private static final ApplicationSubmissionContextReflector instance = - new ApplicationSubmissionContextReflector(ApplicationSubmissionContext.class); - - public static ApplicationSubmissionContextReflector getInstance() { - return instance; - } - - private static final String APPLICATION_TAGS_METHOD_NAME = "setApplicationTags"; - private static final String ATTEMPT_FAILURES_METHOD_NAME = "setAttemptFailuresValidityInterval"; - private static final String KEEP_CONTAINERS_METHOD_NAME = "setKeepContainersAcrossApplicationAttempts"; - private static final String NODE_LABEL_EXPRESSION_NAME = "setNodeLabelExpression"; - - private final Method applicationTagsMethod; - private final Method attemptFailuresValidityIntervalMethod; - private final Method keepContainersMethod; - @Nullable - private final Method nodeLabelExpressionMethod; - - private ApplicationSubmissionContextReflector(Class clazz) { - Method applicationTagsMethod; - Method attemptFailuresValidityIntervalMethod; - Method keepContainersMethod; - Method nodeLabelExpressionMethod; - - try { - // this method is only supported by Hadoop 2.4.0 onwards - applicationTagsMethod = clazz.getMethod(APPLICATION_TAGS_METHOD_NAME, Set.class); - LOG.debug("{} supports method {}.", clazz.getCanonicalName(), APPLICATION_TAGS_METHOD_NAME); - } catch (NoSuchMethodException e) { - LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), APPLICATION_TAGS_METHOD_NAME); - // assign null because the Hadoop version apparently does not support this call. - applicationTagsMethod = null; - } - - this.applicationTagsMethod = applicationTagsMethod; - - try { - // this method is only supported by Hadoop 2.6.0 onwards - attemptFailuresValidityIntervalMethod = clazz.getMethod(ATTEMPT_FAILURES_METHOD_NAME, long.class); - LOG.debug("{} supports method {}.", clazz.getCanonicalName(), ATTEMPT_FAILURES_METHOD_NAME); - } catch (NoSuchMethodException e) { - LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), ATTEMPT_FAILURES_METHOD_NAME); - // assign null because the Hadoop version apparently does not support this call. - attemptFailuresValidityIntervalMethod = null; - } - - this.attemptFailuresValidityIntervalMethod = attemptFailuresValidityIntervalMethod; - - try { - // this method is only supported by Hadoop 2.4.0 onwards - keepContainersMethod = clazz.getMethod(KEEP_CONTAINERS_METHOD_NAME, boolean.class); - LOG.debug("{} supports method {}.", clazz.getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); - } catch (NoSuchMethodException e) { - LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); - // assign null because the Hadoop version apparently does not support this call. - keepContainersMethod = null; - } - - this.keepContainersMethod = keepContainersMethod; - - try { - nodeLabelExpressionMethod = clazz.getMethod(NODE_LABEL_EXPRESSION_NAME, String.class); - LOG.debug("{} supports method {}.", clazz.getCanonicalName(), NODE_LABEL_EXPRESSION_NAME); - } catch (NoSuchMethodException e) { - LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), NODE_LABEL_EXPRESSION_NAME); - nodeLabelExpressionMethod = null; - } - - this.nodeLabelExpressionMethod = nodeLabelExpressionMethod; - } - - public void setApplicationTags( - ApplicationSubmissionContext appContext, - Set applicationTags) throws InvocationTargetException, IllegalAccessException { - if (applicationTagsMethod != null) { - LOG.debug("Calling method {} of {}.", - applicationTagsMethod.getName(), - appContext.getClass().getCanonicalName()); - applicationTagsMethod.invoke(appContext, applicationTags); - } else { - LOG.debug("{} does not support method {}. Doing nothing.", - appContext.getClass().getCanonicalName(), - APPLICATION_TAGS_METHOD_NAME); - } - } - - public void setApplicationNodeLabel( - ApplicationSubmissionContext appContext, - String nodeLabel) throws InvocationTargetException, IllegalAccessException { - if (nodeLabelExpressionMethod != null) { - LOG.debug("Calling method {} of {}.", - nodeLabelExpressionMethod.getName(), - appContext.getClass().getCanonicalName()); - nodeLabelExpressionMethod.invoke(appContext, nodeLabel); - } else { - LOG.debug("{} does not support method {}. Doing nothing.", - appContext.getClass().getCanonicalName(), - NODE_LABEL_EXPRESSION_NAME); - } - } - - public void setAttemptFailuresValidityInterval( - ApplicationSubmissionContext appContext, - long validityInterval) throws InvocationTargetException, IllegalAccessException { - if (attemptFailuresValidityIntervalMethod != null) { - LOG.debug("Calling method {} of {}.", - attemptFailuresValidityIntervalMethod.getName(), - appContext.getClass().getCanonicalName()); - attemptFailuresValidityIntervalMethod.invoke(appContext, validityInterval); - } else { - LOG.debug("{} does not support method {}. Doing nothing.", - appContext.getClass().getCanonicalName(), - ATTEMPT_FAILURES_METHOD_NAME); - } - } - - public void setKeepContainersAcrossApplicationAttempts( - ApplicationSubmissionContext appContext, - boolean keepContainers) throws InvocationTargetException, IllegalAccessException { - - if (keepContainersMethod != null) { - LOG.debug("Calling method {} of {}.", keepContainersMethod.getName(), - appContext.getClass().getCanonicalName()); - keepContainersMethod.invoke(appContext, keepContainers); - } else { - LOG.debug("{} does not support method {}. Doing nothing.", - appContext.getClass().getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); - } - } - } - - private static class YarnDeploymentException extends RuntimeException { - private static final long serialVersionUID = -812040641215388943L; - - public YarnDeploymentException(String message) { - super(message); - } - - public YarnDeploymentException(String message, Throwable cause) { - super(message, cause); - } - } - - private class DeploymentFailureHook extends Thread { - - private final YarnClient yarnClient; - private final YarnClientApplication yarnApplication; - private final Path yarnFilesDir; - - DeploymentFailureHook(YarnClient yarnClient, YarnClientApplication yarnApplication, Path yarnFilesDir) { - this.yarnClient = Preconditions.checkNotNull(yarnClient); - this.yarnApplication = Preconditions.checkNotNull(yarnApplication); - this.yarnFilesDir = Preconditions.checkNotNull(yarnFilesDir); - } - - @Override - public void run() { - LOG.info("Cancelling deployment from Deployment Failure Hook"); - failSessionDuringDeployment(yarnClient, yarnApplication); - LOG.info("Deleting files in {}.", yarnFilesDir); - try { - FileSystem fs = FileSystem.get(yarnConfiguration); - - if (!fs.delete(yarnFilesDir, true)) { - throw new IOException("Deleting files in " + yarnFilesDir + " was unsuccessful"); - } - - fs.close(); - } catch (IOException e) { - LOG.error("Failed to delete Flink Jar and configuration files in HDFS", e); - } - } - } - - protected void addEnvironmentFoldersToShipFiles(Collection effectiveShipFiles) { - addLibFoldersToShipFiles(effectiveShipFiles); - addPluginsFoldersToShipFiles(effectiveShipFiles); - } - - private void addLibFoldersToShipFiles(Collection effectiveShipFiles) { - // Add lib folder to the ship files if the environment variable is set. - // This is for convenience when running from the command-line. - // (for other files users explicitly set the ship files) - String libDir = System.getenv().get(ENV_FLINK_LIB_DIR); - if (libDir != null) { - File directoryFile = new File(libDir); - if (directoryFile.isDirectory()) { - effectiveShipFiles.add(directoryFile); - } else { - throw new YarnDeploymentException("The environment variable '" + ENV_FLINK_LIB_DIR + - "' is set to '" + libDir + "' but the directory doesn't exist."); - } - } else if (this.shipFiles.isEmpty()) { - LOG.warn("Environment variable '{}' not set and ship files have not been provided manually. " + - "Not shipping any library files.", ENV_FLINK_LIB_DIR); - } - } - - private void addPluginsFoldersToShipFiles(Collection effectiveShipFiles) { - String pluginsDir = System.getenv().get(ENV_FLINK_PLUGINS_DIR); - if (pluginsDir != null) { - File directoryFile = new File(pluginsDir); - if (directoryFile.isDirectory()) { - effectiveShipFiles.add(directoryFile); - } else { - LOG.warn("The environment variable '" + ENV_FLINK_PLUGINS_DIR + - "' is set to '" + pluginsDir + "' but the directory doesn't exist."); - } - } - } - - protected ContainerLaunchContext setupApplicationMasterContainer( - String yarnClusterEntrypoint, - boolean hasLogback, - boolean hasLog4j, - boolean hasKrb5, - int jobManagerMemoryMb) { - // ------------------ Prepare Application Master Container ------------------------------ - - // respect custom JVM options in the YAML file - String javaOpts = flinkConfiguration.getString(CoreOptions.FLINK_JVM_OPTIONS); - if (flinkConfiguration.getString(CoreOptions.FLINK_JM_JVM_OPTIONS).length() > 0) { - javaOpts += " " + flinkConfiguration.getString(CoreOptions.FLINK_JM_JVM_OPTIONS); - } - //applicable only for YarnMiniCluster secure test run - //krb5.conf file will be available as local resource in JM/TM container - if (hasKrb5) { - javaOpts += " -Djava.security.krb5.conf=krb5.conf"; - } - - // Set up the container launch context for the application master - ContainerLaunchContext amContainer = Records.newRecord(ContainerLaunchContext.class); - - final Map startCommandValues = new HashMap<>(); - startCommandValues.put("java", "$JAVA_HOME/bin/java"); - - int heapSize = Utils.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration); - String jvmHeapMem = String.format("-Xms%sm -Xmx%sm", heapSize, heapSize); - startCommandValues.put("jvmmem", jvmHeapMem); - - startCommandValues.put("jvmopts", javaOpts); - String logging = ""; - - if (hasLogback || hasLog4j) { - logging = "-Dlog.file=\"" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.log\""; - - if (hasLogback) { - logging += " -Dlogback.configurationFile=file:" + CONFIG_FILE_LOGBACK_NAME; - } - - if (hasLog4j) { - logging += " -Dlog4j.configuration=file:" + CONFIG_FILE_LOG4J_NAME; - } - } - - startCommandValues.put("logging", logging); - startCommandValues.put("class", yarnClusterEntrypoint); - startCommandValues.put("redirects", - "1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.out " + - "2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.err"); - startCommandValues.put("args", ""); - - final String commandTemplate = flinkConfiguration - .getString(ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE, - ConfigConstants.DEFAULT_YARN_CONTAINER_START_COMMAND_TEMPLATE); - final String amCommand = - BootstrapTools.getStartCommand(commandTemplate, startCommandValues); - - amContainer.setCommands(Collections.singletonList(amCommand)); - - LOG.debug("Application Master start command: " + amCommand); - - return amContainer; - } - - private static YarnConfigOptions.UserJarInclusion getUserJarInclusionMode(org.apache.flink.configuration.Configuration config) { - throwIfUserTriesToDisableUserJarInclusionInSystemClassPath(config); - - return config.getEnum(YarnConfigOptions.UserJarInclusion.class, YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR); - } - - private static void throwIfUserTriesToDisableUserJarInclusionInSystemClassPath(final Configuration config) { - final String userJarInclusion = config.getString(YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR); - if ("DISABLED".equalsIgnoreCase(userJarInclusion)) { - throw new IllegalArgumentException(String.format("Config option %s cannot be set to DISABLED anymore (see FLINK-11781)", - YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR.key())); - } - } - - /** - * Creates a YarnClusterClient; may be overridden in tests. - */ - protected abstract ClusterClient createYarnClusterClient( - AbstractYarnClusterDescriptor descriptor, - int numberTaskManagers, - int slotsPerTaskManager, - ApplicationReport report, - org.apache.flink.configuration.Configuration flinkConfiguration, - boolean perJobCluster) throws Exception; -} - diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index 3dff72fb8c..b1782416fa 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -18,25 +18,135 @@ package org.apache.flink.yarn; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.client.deployment.ClusterDeploymentException; +import org.apache.flink.client.deployment.ClusterDescriptor; +import org.apache.flink.client.deployment.ClusterRetrieveException; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.ResourceManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.SecurityOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.runtime.clusterframework.BootstrapTools; +import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; +import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.runtime.taskexecutor.TaskManagerServices; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ShutdownHookUtil; +import org.apache.flink.yarn.configuration.YarnConfigOptions; import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.NodeState; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.QueueInfo; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.util.Records; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.ObjectOutputStream; +import java.io.PrintStream; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_LIB_DIR; +import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_PLUGINS_DIR; +import static org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever.JOB_GRAPH_FILE_PATH; +import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.CONFIG_FILE_LOG4J_NAME; +import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.CONFIG_FILE_LOGBACK_NAME; +import static org.apache.flink.yarn.cli.FlinkYarnSessionCli.getDynamicProperties; /** - * Implementation of {@link AbstractYarnClusterDescriptor} which is used to start the - * application master. + * The descriptor with deployment information for deploying a Flink cluster on Yarn. */ -public class YarnClusterDescriptor extends AbstractYarnClusterDescriptor { +public class YarnClusterDescriptor implements ClusterDescriptor { + private static final Logger LOG = LoggerFactory.getLogger(YarnClusterDescriptor.class); + + private final YarnConfiguration yarnConfiguration; + + private final YarnClient yarnClient; + + /** True if the descriptor must not shut down the YarnClient. */ + private final boolean sharedYarnClient; + + private String yarnQueue; + + private String configurationDirectory; + + private Path flinkJarPath; + + private String dynamicPropertiesEncoded; + + /** Lazily initialized list of files to ship. */ + protected List shipFiles = new LinkedList<>(); + + private final Configuration flinkConfiguration; + + private boolean detached; + + private String customName; + + private String zookeeperNamespace; + + private String nodeLabel; + + private String applicationType; + + private YarnConfigOptions.UserJarInclusion userJarInclusion; public YarnClusterDescriptor( Configuration flinkConfiguration, @@ -44,24 +154,244 @@ public class YarnClusterDescriptor extends AbstractYarnClusterDescriptor { String configurationDirectory, YarnClient yarnClient, boolean sharedYarnClient) { - super( - flinkConfiguration, - yarnConfiguration, - configurationDirectory, - yarnClient, - sharedYarnClient); + + this.yarnConfiguration = Preconditions.checkNotNull(yarnConfiguration); + this.yarnClient = Preconditions.checkNotNull(yarnClient); + this.sharedYarnClient = sharedYarnClient; + + this.flinkConfiguration = Preconditions.checkNotNull(flinkConfiguration); + userJarInclusion = getUserJarInclusionMode(flinkConfiguration); + + this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory); } - @Override + public YarnClient getYarnClient() { + return yarnClient; + } + + /** + * The class to start the application master with. This class runs the main + * method in case of session cluster. + */ protected String getYarnSessionClusterEntrypoint() { return YarnSessionClusterEntrypoint.class.getName(); } - @Override + /** + * The class to start the application master with. This class runs the main + * method in case of the job cluster. + */ protected String getYarnJobClusterEntrypoint() { return YarnJobClusterEntrypoint.class.getName(); } + public Configuration getFlinkConfiguration() { + return flinkConfiguration; + } + + public void setQueue(String queue) { + this.yarnQueue = queue; + } + + public void setLocalJarPath(Path localJarPath) { + if (!localJarPath.toString().endsWith("jar")) { + throw new IllegalArgumentException("The passed jar path ('" + localJarPath + "') does not end with the 'jar' extension"); + } + this.flinkJarPath = localJarPath; + } + + /** + * Adds the given files to the list of files to ship. + * + *

Note that any file matching "flink-dist*.jar" will be excluded from the upload by + * {@link #uploadAndRegisterFiles(Collection, FileSystem, Path, ApplicationId, List, Map, StringBuilder)} + * since we upload the Flink uber jar ourselves and do not need to deploy it multiple times. + * + * @param shipFiles files to ship + */ + public void addShipFiles(List shipFiles) { + this.shipFiles.addAll(shipFiles); + } + + public void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded) { + this.dynamicPropertiesEncoded = dynamicPropertiesEncoded; + } + + public String getDynamicPropertiesEncoded() { + return this.dynamicPropertiesEncoded; + } + + private void isReadyForDeployment(ClusterSpecification clusterSpecification) throws YarnDeploymentException { + + if (clusterSpecification.getNumberTaskManagers() <= 0) { + throw new YarnDeploymentException("Taskmanager count must be positive"); + } + if (this.flinkJarPath == null) { + throw new YarnDeploymentException("The Flink jar path is null"); + } + if (this.configurationDirectory == null) { + throw new YarnDeploymentException("Configuration directory not set"); + } + if (this.flinkConfiguration == null) { + throw new YarnDeploymentException("Flink configuration object has not been set"); + } + + // Check if we don't exceed YARN's maximum virtual cores. + // Fetch numYarnMaxVcores from all the RUNNING nodes via yarnClient + final int numYarnMaxVcores; + try { + numYarnMaxVcores = yarnClient.getNodeReports(NodeState.RUNNING) + .stream() + .mapToInt(report -> report.getCapability().getVirtualCores()) + .max() + .orElse(0); + } catch (Exception e) { + throw new YarnDeploymentException("Couldn't get cluster description, please check on the YarnConfiguration", e); + } + + int configuredAmVcores = flinkConfiguration.getInteger(YarnConfigOptions.APP_MASTER_VCORES); + if (configuredAmVcores > numYarnMaxVcores) { + throw new IllegalConfigurationException( + String.format("The number of requested virtual cores for application master %d" + + " exceeds the maximum number of virtual cores %d available in the Yarn Cluster.", + configuredAmVcores, numYarnMaxVcores)); + } + + int configuredVcores = flinkConfiguration.getInteger(YarnConfigOptions.VCORES, clusterSpecification.getSlotsPerTaskManager()); + // don't configure more than the maximum configured number of vcores + if (configuredVcores > numYarnMaxVcores) { + throw new IllegalConfigurationException( + String.format("The number of requested virtual cores per node %d" + + " exceeds the maximum number of virtual cores %d available in the Yarn Cluster." + + " Please note that the number of virtual cores is set to the number of task slots by default" + + " unless configured in the Flink config with '%s.'", + configuredVcores, numYarnMaxVcores, YarnConfigOptions.VCORES.key())); + } + + // check if required Hadoop environment variables are set. If not, warn user + if (System.getenv("HADOOP_CONF_DIR") == null && + System.getenv("YARN_CONF_DIR") == null) { + LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set. " + + "The Flink YARN Client needs one of these to be set to properly load the Hadoop " + + "configuration for accessing YARN."); + } + } + + private static boolean allocateResource(int[] nodeManagers, int toAllocate) { + for (int i = 0; i < nodeManagers.length; i++) { + if (nodeManagers[i] >= toAllocate) { + nodeManagers[i] -= toAllocate; + return true; + } + } + return false; + } + + /** + * @deprecated The cluster descriptor should not know about this option. + */ + @Deprecated + public void setDetachedMode(boolean detachedMode) { + this.detached = detachedMode; + } + + /** + * @deprecated The cluster descriptor should not know about this option. + */ + @Deprecated + public boolean isDetachedMode() { + return detached; + } + + public String getZookeeperNamespace() { + return zookeeperNamespace; + } + + public void setZookeeperNamespace(String zookeeperNamespace) { + this.zookeeperNamespace = zookeeperNamespace; + } + + public String getNodeLabel() { + return nodeLabel; + } + + public void setNodeLabel(String nodeLabel) { + this.nodeLabel = nodeLabel; + } + + // ------------------------------------------------------------- + // Lifecycle management + // ------------------------------------------------------------- + + @Override + public void close() { + if (!sharedYarnClient) { + yarnClient.stop(); + } + } + + // ------------------------------------------------------------- + // ClusterClient overrides + // ------------------------------------------------------------- + + @Override + public ClusterClient retrieve(ApplicationId applicationId) throws ClusterRetrieveException { + + try { + // check if required Hadoop environment variables are set. If not, warn user + if (System.getenv("HADOOP_CONF_DIR") == null && + System.getenv("YARN_CONF_DIR") == null) { + LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set." + + "The Flink YARN Client needs one of these to be set to properly load the Hadoop " + + "configuration for accessing YARN."); + } + + final ApplicationReport report = yarnClient.getApplicationReport(applicationId); + + if (report.getFinalApplicationStatus() != FinalApplicationStatus.UNDEFINED) { + // Flink cluster is not running anymore + LOG.error("The application {} doesn't run anymore. It has previously completed with final status: {}", + applicationId, report.getFinalApplicationStatus()); + throw new RuntimeException("The Yarn application " + applicationId + " doesn't run anymore."); + } + + final String host = report.getHost(); + final int port = report.getRpcPort(); + + LOG.info("Found Web Interface {}:{} of application '{}'.", host, port, applicationId); + + flinkConfiguration.setString(JobManagerOptions.ADDRESS, host); + flinkConfiguration.setInteger(JobManagerOptions.PORT, port); + + flinkConfiguration.setString(RestOptions.ADDRESS, host); + flinkConfiguration.setInteger(RestOptions.PORT, port); + + return createYarnClusterClient( + this, + -1, // we don't know the number of task managers of a started Flink cluster + -1, // we don't know how many slots each task manager has for a started Flink cluster + report, + flinkConfiguration, + false); + } catch (Exception e) { + throw new ClusterRetrieveException("Couldn't retrieve Yarn cluster", e); + } + } + + @Override + public ClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) throws ClusterDeploymentException { + try { + return deployInternal( + clusterSpecification, + "Flink session cluster", + getYarnSessionClusterEntrypoint(), + null, + false); + } catch (Exception e) { + throw new ClusterDeploymentException("Couldn't deploy Yarn session cluster", e); + } + } + @Override public ClusterClient deployJobCluster( ClusterSpecification clusterSpecification, @@ -84,15 +414,1271 @@ public class YarnClusterDescriptor extends AbstractYarnClusterDescriptor { } @Override + public void killCluster(ApplicationId applicationId) throws FlinkException { + try { + yarnClient.killApplication(applicationId); + Utils.deleteApplicationFiles(Collections.singletonMap( + YarnConfigKeys.FLINK_YARN_FILES, + getYarnFilesDir(applicationId).toUri().toString())); + } catch (YarnException | IOException e) { + throw new FlinkException("Could not kill the Yarn Flink cluster with id " + applicationId + '.', e); + } + } + + /** + * Method to validate cluster specification before deploy it, it will throw + * an {@link FlinkException} if the {@link ClusterSpecification} is invalid. + * + * @param clusterSpecification cluster specification to check against the configuration of the + * YarnClusterDescriptor + * @throws FlinkException if the cluster cannot be started with the provided {@link ClusterSpecification} + */ + private void validateClusterSpecification(ClusterSpecification clusterSpecification) throws FlinkException { + try { + final long taskManagerMemorySize = clusterSpecification.getTaskManagerMemoryMB(); + // We do the validation by calling the calculation methods here + // Internally these methods will check whether the cluster can be started with the provided + // ClusterSpecification and the configured memory requirements + final long cutoff = ContaineredTaskManagerParameters.calculateCutoffMB(flinkConfiguration, taskManagerMemorySize); + TaskManagerServices.calculateHeapSizeMB(taskManagerMemorySize - cutoff, flinkConfiguration); + } catch (IllegalArgumentException iae) { + throw new FlinkException("Cannot fulfill the minimum memory requirements with the provided " + + "cluster specification. Please increase the memory of the cluster.", iae); + } + } + + /** + * This method will block until the ApplicationMaster/JobManager have been deployed on YARN. + * + * @param clusterSpecification Initial cluster specification for the Flink cluster to be deployed + * @param applicationName name of the Yarn application to start + * @param yarnClusterEntrypoint Class name of the Yarn cluster entry point. + * @param jobGraph A job graph which is deployed with the Flink cluster, {@code null} if none + * @param detached True if the cluster should be started in detached mode + */ + protected ClusterClient deployInternal( + ClusterSpecification clusterSpecification, + String applicationName, + String yarnClusterEntrypoint, + @Nullable JobGraph jobGraph, + boolean detached) throws Exception { + + // ------------------ Check if configuration is valid -------------------- + validateClusterSpecification(clusterSpecification); + + if (UserGroupInformation.isSecurityEnabled()) { + // note: UGI::hasKerberosCredentials inaccurately reports false + // for logins based on a keytab (fixed in Hadoop 2.6.1, see HADOOP-10786), + // so we check only in ticket cache scenario. + boolean useTicketCache = flinkConfiguration.getBoolean(SecurityOptions.KERBEROS_LOGIN_USETICKETCACHE); + + UserGroupInformation loginUser = UserGroupInformation.getCurrentUser(); + if (loginUser.getAuthenticationMethod() == UserGroupInformation.AuthenticationMethod.KERBEROS + && useTicketCache && !loginUser.hasKerberosCredentials()) { + LOG.error("Hadoop security with Kerberos is enabled but the login user does not have Kerberos credentials"); + throw new RuntimeException("Hadoop security with Kerberos is enabled but the login user " + + "does not have Kerberos credentials"); + } + } + + isReadyForDeployment(clusterSpecification); + + // ------------------ Check if the specified queue exists -------------------- + + checkYarnQueues(yarnClient); + + // ------------------ Add dynamic properties to local flinkConfiguraton ------ + Map dynProperties = getDynamicProperties(dynamicPropertiesEncoded); + for (Map.Entry dynProperty : dynProperties.entrySet()) { + flinkConfiguration.setString(dynProperty.getKey(), dynProperty.getValue()); + } + + // ------------------ Check if the YARN ClusterClient has the requested resources -------------- + + // Create application via yarnClient + final YarnClientApplication yarnApplication = yarnClient.createApplication(); + final GetNewApplicationResponse appResponse = yarnApplication.getNewApplicationResponse(); + + Resource maxRes = appResponse.getMaximumResourceCapability(); + + final ClusterResourceDescription freeClusterMem; + try { + freeClusterMem = getCurrentFreeClusterResources(yarnClient); + } catch (YarnException | IOException e) { + failSessionDuringDeployment(yarnClient, yarnApplication); + throw new YarnDeploymentException("Could not retrieve information about free cluster resources.", e); + } + + final int yarnMinAllocationMB = yarnConfiguration.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0); + + final ClusterSpecification validClusterSpecification; + try { + validClusterSpecification = validateClusterResources( + clusterSpecification, + yarnMinAllocationMB, + maxRes, + freeClusterMem); + } catch (YarnDeploymentException yde) { + failSessionDuringDeployment(yarnClient, yarnApplication); + throw yde; + } + + LOG.info("Cluster specification: {}", validClusterSpecification); + + final ClusterEntrypoint.ExecutionMode executionMode = detached ? + ClusterEntrypoint.ExecutionMode.DETACHED + : ClusterEntrypoint.ExecutionMode.NORMAL; + + flinkConfiguration.setString(ClusterEntrypoint.EXECUTION_MODE, executionMode.toString()); + + ApplicationReport report = startAppMaster( + flinkConfiguration, + applicationName, + yarnClusterEntrypoint, + jobGraph, + yarnClient, + yarnApplication, + validClusterSpecification); + + final String host = report.getHost(); + final int port = report.getRpcPort(); + + flinkConfiguration.setString(JobManagerOptions.ADDRESS, host); + flinkConfiguration.setInteger(JobManagerOptions.PORT, port); + + flinkConfiguration.setString(RestOptions.ADDRESS, host); + flinkConfiguration.setInteger(RestOptions.PORT, port); + + // the Flink cluster is deployed in YARN. Represent cluster + return createYarnClusterClient( + this, + validClusterSpecification.getNumberTaskManagers(), + validClusterSpecification.getSlotsPerTaskManager(), + report, + flinkConfiguration, + true); + } + + protected ClusterSpecification validateClusterResources( + ClusterSpecification clusterSpecification, + int yarnMinAllocationMB, + Resource maximumResourceCapability, + ClusterResourceDescription freeClusterResources) throws YarnDeploymentException { + + int taskManagerCount = clusterSpecification.getNumberTaskManagers(); + int jobManagerMemoryMb = clusterSpecification.getMasterMemoryMB(); + int taskManagerMemoryMb = clusterSpecification.getTaskManagerMemoryMB(); + + if (jobManagerMemoryMb < yarnMinAllocationMB || taskManagerMemoryMb < yarnMinAllocationMB) { + LOG.warn("The JobManager or TaskManager memory is below the smallest possible YARN Container size. " + + "The value of 'yarn.scheduler.minimum-allocation-mb' is '" + yarnMinAllocationMB + "'. Please increase the memory size." + + "YARN will allocate the smaller containers but the scheduler will account for the minimum-allocation-mb, maybe not all instances " + + "you requested will start."); + } + + // set the memory to minAllocationMB to do the next checks correctly + if (jobManagerMemoryMb < yarnMinAllocationMB) { + jobManagerMemoryMb = yarnMinAllocationMB; + } + if (taskManagerMemoryMb < yarnMinAllocationMB) { + taskManagerMemoryMb = yarnMinAllocationMB; + } + + final String note = "Please check the 'yarn.scheduler.maximum-allocation-mb' and the 'yarn.nodemanager.resource.memory-mb' configuration values\n"; + if (jobManagerMemoryMb > maximumResourceCapability.getMemory()) { + throw new YarnDeploymentException("The cluster does not have the requested resources for the JobManager available!\n" + + "Maximum Memory: " + maximumResourceCapability.getMemory() + "MB Requested: " + jobManagerMemoryMb + "MB. " + note); + } + + if (taskManagerMemoryMb > maximumResourceCapability.getMemory()) { + throw new YarnDeploymentException("The cluster does not have the requested resources for the TaskManagers available!\n" + + "Maximum Memory: " + maximumResourceCapability.getMemory() + " Requested: " + taskManagerMemoryMb + "MB. " + note); + } + + final String noteRsc = "\nThe Flink YARN client will try to allocate the YARN session, but maybe not all TaskManagers are " + + "connecting from the beginning because the resources are currently not available in the cluster. " + + "The allocation might take more time than usual because the Flink YARN client needs to wait until " + + "the resources become available."; + int totalMemoryRequired = jobManagerMemoryMb + taskManagerMemoryMb * taskManagerCount; + + if (freeClusterResources.totalFreeMemory < totalMemoryRequired) { + LOG.warn("This YARN session requires " + totalMemoryRequired + "MB of memory in the cluster. " + + "There are currently only " + freeClusterResources.totalFreeMemory + "MB available." + noteRsc); + + } + if (taskManagerMemoryMb > freeClusterResources.containerLimit) { + LOG.warn("The requested amount of memory for the TaskManagers (" + taskManagerMemoryMb + "MB) is more than " + + "the largest possible YARN container: " + freeClusterResources.containerLimit + noteRsc); + } + if (jobManagerMemoryMb > freeClusterResources.containerLimit) { + LOG.warn("The requested amount of memory for the JobManager (" + jobManagerMemoryMb + "MB) is more than " + + "the largest possible YARN container: " + freeClusterResources.containerLimit + noteRsc); + } + + // ----------------- check if the requested containers fit into the cluster. + + int[] nmFree = Arrays.copyOf(freeClusterResources.nodeManagersFree, freeClusterResources.nodeManagersFree.length); + // first, allocate the jobManager somewhere. + if (!allocateResource(nmFree, jobManagerMemoryMb)) { + LOG.warn("Unable to find a NodeManager that can fit the JobManager/Application master. " + + "The JobManager requires " + jobManagerMemoryMb + "MB. NodeManagers available: " + + Arrays.toString(freeClusterResources.nodeManagersFree) + noteRsc); + } + // allocate TaskManagers + for (int i = 0; i < taskManagerCount; i++) { + if (!allocateResource(nmFree, taskManagerMemoryMb)) { + LOG.warn("There is not enough memory available in the YARN cluster. " + + "The TaskManager(s) require " + taskManagerMemoryMb + "MB each. " + + "NodeManagers available: " + Arrays.toString(freeClusterResources.nodeManagersFree) + "\n" + + "After allocating the JobManager (" + jobManagerMemoryMb + "MB) and (" + i + "/" + taskManagerCount + ") TaskManagers, " + + "the following NodeManagers are available: " + Arrays.toString(nmFree) + noteRsc); + } + } + + return new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(jobManagerMemoryMb) + .setTaskManagerMemoryMB(taskManagerMemoryMb) + .setNumberTaskManagers(clusterSpecification.getNumberTaskManagers()) + .setSlotsPerTaskManager(clusterSpecification.getSlotsPerTaskManager()) + .createClusterSpecification(); + + } + + private void checkYarnQueues(YarnClient yarnClient) { + try { + List queues = yarnClient.getAllQueues(); + if (queues.size() > 0 && this.yarnQueue != null) { // check only if there are queues configured in yarn and for this session. + boolean queueFound = false; + for (QueueInfo queue : queues) { + if (queue.getQueueName().equals(this.yarnQueue)) { + queueFound = true; + break; + } + } + if (!queueFound) { + String queueNames = ""; + for (QueueInfo queue : queues) { + queueNames += queue.getQueueName() + ", "; + } + LOG.warn("The specified queue '" + this.yarnQueue + "' does not exist. " + + "Available queues: " + queueNames); + } + } else { + LOG.debug("The YARN cluster does not have any queues configured"); + } + } catch (Throwable e) { + LOG.warn("Error while getting queue information from YARN: " + e.getMessage()); + if (LOG.isDebugEnabled()) { + LOG.debug("Error details", e); + } + } + } + + public ApplicationReport startAppMaster( + Configuration configuration, + String applicationName, + String yarnClusterEntrypoint, + JobGraph jobGraph, + YarnClient yarnClient, + YarnClientApplication yarnApplication, + ClusterSpecification clusterSpecification) throws Exception { + + // ------------------ Initialize the file systems ------------------------- + + org.apache.flink.core.fs.FileSystem.initialize( + configuration, + PluginUtils.createPluginManagerFromRootFolder(configuration)); + + // initialize file system + // Copy the application master jar to the filesystem + // Create a local resource to point to the destination jar path + final FileSystem fs = FileSystem.get(yarnConfiguration); + final Path homeDir = fs.getHomeDirectory(); + + // hard coded check for the GoogleHDFS client because its not overriding the getScheme() method. + if (!fs.getClass().getSimpleName().equals("GoogleHadoopFileSystem") && + fs.getScheme().startsWith("file")) { + LOG.warn("The file system scheme is '" + fs.getScheme() + "'. This indicates that the " + + "specified Hadoop configuration path is wrong and the system is using the default Hadoop configuration values." + + "The Flink YARN client needs to store its files in a distributed file system"); + } + + ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext(); + Set systemShipFiles = new HashSet<>(shipFiles.size()); + for (File file : shipFiles) { + systemShipFiles.add(file.getAbsoluteFile()); + } + + //check if there is a logback or log4j file + File logbackFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOGBACK_NAME); + final boolean hasLogback = logbackFile.exists(); + if (hasLogback) { + systemShipFiles.add(logbackFile); + } + + File log4jFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOG4J_NAME); + final boolean hasLog4j = log4jFile.exists(); + if (hasLog4j) { + systemShipFiles.add(log4jFile); + if (hasLogback) { + // this means there is already a logback configuration file --> fail + LOG.warn("The configuration directory ('" + configurationDirectory + "') contains both LOG4J and " + + "Logback configuration files. Please delete or rename one of them."); + } + } + + addEnvironmentFoldersToShipFiles(systemShipFiles); + + // Set-up ApplicationSubmissionContext for the application + + final ApplicationId appId = appContext.getApplicationId(); + + // ------------------ Add Zookeeper namespace to local flinkConfiguraton ------ + String zkNamespace = getZookeeperNamespace(); + // no user specified cli argument for namespace? + if (zkNamespace == null || zkNamespace.isEmpty()) { + // namespace defined in config? else use applicationId as default. + zkNamespace = configuration.getString(HighAvailabilityOptions.HA_CLUSTER_ID, String.valueOf(appId)); + setZookeeperNamespace(zkNamespace); + } + + configuration.setString(HighAvailabilityOptions.HA_CLUSTER_ID, zkNamespace); + + if (HighAvailabilityMode.isHighAvailabilityModeActivated(configuration)) { + // activate re-execution of failed applications + appContext.setMaxAppAttempts( + configuration.getInteger( + YarnConfigOptions.APPLICATION_ATTEMPTS.key(), + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)); + + activateHighAvailabilitySupport(appContext); + } else { + // set number of application retries to 1 in the default case + appContext.setMaxAppAttempts( + configuration.getInteger( + YarnConfigOptions.APPLICATION_ATTEMPTS.key(), + 1)); + } + + final Set userJarFiles = (jobGraph == null) + // not per-job submission + ? Collections.emptySet() + // add user code jars from the provided JobGraph + : jobGraph.getUserJars().stream().map(f -> f.toUri()).map(File::new).collect(Collectors.toSet()); + + // local resource map for Yarn + final Map localResources = new HashMap<>(2 + systemShipFiles.size() + userJarFiles.size()); + // list of remote paths (after upload) + final List paths = new ArrayList<>(2 + systemShipFiles.size() + userJarFiles.size()); + // ship list that enables reuse of resources for task manager containers + StringBuilder envShipFileList = new StringBuilder(); + + // upload and register ship files + List systemClassPaths = uploadAndRegisterFiles( + systemShipFiles, + fs, + homeDir, + appId, + paths, + localResources, + envShipFileList); + + final List userClassPaths = uploadAndRegisterFiles( + userJarFiles, + fs, + homeDir, + appId, + paths, + localResources, + envShipFileList); + + if (userJarInclusion == YarnConfigOptions.UserJarInclusion.ORDER) { + systemClassPaths.addAll(userClassPaths); + } + + // normalize classpath by sorting + Collections.sort(systemClassPaths); + Collections.sort(userClassPaths); + + // classpath assembler + StringBuilder classPathBuilder = new StringBuilder(); + if (userJarInclusion == YarnConfigOptions.UserJarInclusion.FIRST) { + for (String userClassPath : userClassPaths) { + classPathBuilder.append(userClassPath).append(File.pathSeparator); + } + } + for (String classPath : systemClassPaths) { + classPathBuilder.append(classPath).append(File.pathSeparator); + } + + // Setup jar for ApplicationMaster + Path remotePathJar = setupSingleLocalResource( + "flink.jar", + fs, + appId, + flinkJarPath, + localResources, + homeDir, + ""); + + // set the right configuration values for the TaskManager + configuration.setInteger( + TaskManagerOptions.NUM_TASK_SLOTS, + clusterSpecification.getSlotsPerTaskManager()); + + configuration.setString( + TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, + clusterSpecification.getTaskManagerMemoryMB() + "m"); + + // Upload the flink configuration + // write out configuration file + File tmpConfigurationFile = File.createTempFile(appId + "-flink-conf.yaml", null); + tmpConfigurationFile.deleteOnExit(); + BootstrapTools.writeConfiguration(configuration, tmpConfigurationFile); + + Path remotePathConf = setupSingleLocalResource( + "flink-conf.yaml", + fs, + appId, + new Path(tmpConfigurationFile.getAbsolutePath()), + localResources, + homeDir, + ""); + + paths.add(remotePathJar); + classPathBuilder.append("flink.jar").append(File.pathSeparator); + paths.add(remotePathConf); + classPathBuilder.append("flink-conf.yaml").append(File.pathSeparator); + + if (userJarInclusion == YarnConfigOptions.UserJarInclusion.LAST) { + for (String userClassPath : userClassPaths) { + classPathBuilder.append(userClassPath).append(File.pathSeparator); + } + } + + // write job graph to tmp file and add it to local resource + // TODO: server use user main method to generate job graph + if (jobGraph != null) { + try { + File fp = File.createTempFile(appId.toString(), null); + fp.deleteOnExit(); + try (FileOutputStream output = new FileOutputStream(fp); + ObjectOutputStream obOutput = new ObjectOutputStream(output);){ + obOutput.writeObject(jobGraph); + } + + final String jobGraphFilename = "job.graph"; + flinkConfiguration.setString(JOB_GRAPH_FILE_PATH, jobGraphFilename); + + Path pathFromYarnURL = setupSingleLocalResource( + jobGraphFilename, + fs, + appId, + new Path(fp.toURI()), + localResources, + homeDir, + ""); + paths.add(pathFromYarnURL); + classPathBuilder.append(jobGraphFilename).append(File.pathSeparator); + } catch (Exception e) { + LOG.warn("Add job graph to local resource fail"); + throw e; + } + } + + final Path yarnFilesDir = getYarnFilesDir(appId); + FsPermission permission = new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE); + fs.setPermission(yarnFilesDir, permission); // set permission for path. + + //To support Yarn Secure Integration Test Scenario + //In Integration test setup, the Yarn containers created by YarnMiniCluster does not have the Yarn site XML + //and KRB5 configuration files. We are adding these files as container local resources for the container + //applications (JM/TMs) to have proper secure cluster setup + Path remoteKrb5Path = null; + Path remoteYarnSiteXmlPath = null; + boolean hasKrb5 = false; + if (System.getenv("IN_TESTS") != null) { + File f = new File(System.getenv("YARN_CONF_DIR"), Utils.YARN_SITE_FILE_NAME); + LOG.info("Adding Yarn configuration {} to the AM container local resource bucket", f.getAbsolutePath()); + Path yarnSitePath = new Path(f.getAbsolutePath()); + remoteYarnSiteXmlPath = setupSingleLocalResource( + Utils.YARN_SITE_FILE_NAME, + fs, + appId, + yarnSitePath, + localResources, + homeDir, + ""); + + String krb5Config = System.getProperty("java.security.krb5.conf"); + if (krb5Config != null && krb5Config.length() != 0) { + File krb5 = new File(krb5Config); + LOG.info("Adding KRB5 configuration {} to the AM container local resource bucket", krb5.getAbsolutePath()); + Path krb5ConfPath = new Path(krb5.getAbsolutePath()); + remoteKrb5Path = setupSingleLocalResource( + Utils.KRB5_FILE_NAME, + fs, + appId, + krb5ConfPath, + localResources, + homeDir, + ""); + hasKrb5 = true; + } + } + + // setup security tokens + Path remotePathKeytab = null; + String keytab = configuration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB); + if (keytab != null) { + LOG.info("Adding keytab {} to the AM container local resource bucket", keytab); + remotePathKeytab = setupSingleLocalResource( + Utils.KEYTAB_FILE_NAME, + fs, + appId, + new Path(keytab), + localResources, + homeDir, + ""); + } + + final ContainerLaunchContext amContainer = setupApplicationMasterContainer( + yarnClusterEntrypoint, + hasLogback, + hasLog4j, + hasKrb5, + clusterSpecification.getMasterMemoryMB()); + + if (UserGroupInformation.isSecurityEnabled()) { + // set HDFS delegation tokens when security is enabled + LOG.info("Adding delegation token to the AM container.."); + Utils.setTokensFor(amContainer, paths, yarnConfiguration); + } + + amContainer.setLocalResources(localResources); + fs.close(); + + // Setup CLASSPATH and environment variables for ApplicationMaster + final Map appMasterEnv = new HashMap<>(); + // set user specified app master environment variables + appMasterEnv.putAll(Utils.getEnvironmentVariables(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, configuration)); + // set Flink app class path + appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, classPathBuilder.toString()); + + // set Flink on YARN internal configuration values + appMasterEnv.put(YarnConfigKeys.ENV_TM_COUNT, String.valueOf(clusterSpecification.getNumberTaskManagers())); + appMasterEnv.put(YarnConfigKeys.ENV_TM_MEMORY, String.valueOf(clusterSpecification.getTaskManagerMemoryMB())); + appMasterEnv.put(YarnConfigKeys.FLINK_JAR_PATH, remotePathJar.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, homeDir.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, envShipFileList.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_SLOTS, String.valueOf(clusterSpecification.getSlotsPerTaskManager())); + appMasterEnv.put(YarnConfigKeys.ENV_DETACHED, String.valueOf(detached)); + appMasterEnv.put(YarnConfigKeys.ENV_ZOOKEEPER_NAMESPACE, getZookeeperNamespace()); + appMasterEnv.put(YarnConfigKeys.FLINK_YARN_FILES, yarnFilesDir.toUri().toString()); + + // https://github.com/apache/hadoop/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md#identity-on-an-insecure-cluster-hadoop_user_name + appMasterEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, UserGroupInformation.getCurrentUser().getUserName()); + + if (remotePathKeytab != null) { + appMasterEnv.put(YarnConfigKeys.KEYTAB_PATH, remotePathKeytab.toString()); + String principal = configuration.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL); + appMasterEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, principal); + } + + //To support Yarn Secure Integration Test Scenario + if (remoteYarnSiteXmlPath != null) { + appMasterEnv.put(YarnConfigKeys.ENV_YARN_SITE_XML_PATH, remoteYarnSiteXmlPath.toString()); + } + if (remoteKrb5Path != null) { + appMasterEnv.put(YarnConfigKeys.ENV_KRB5_PATH, remoteKrb5Path.toString()); + } + + if (dynamicPropertiesEncoded != null) { + appMasterEnv.put(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES, dynamicPropertiesEncoded); + } + + // set classpath from YARN configuration + Utils.setupYarnClassPath(yarnConfiguration, appMasterEnv); + + amContainer.setEnvironment(appMasterEnv); + + // Set up resource type requirements for ApplicationMaster + Resource capability = Records.newRecord(Resource.class); + capability.setMemory(clusterSpecification.getMasterMemoryMB()); + capability.setVirtualCores(flinkConfiguration.getInteger(YarnConfigOptions.APP_MASTER_VCORES)); + + final String customApplicationName = customName != null ? customName : applicationName; + + appContext.setApplicationName(customApplicationName); + appContext.setApplicationType(applicationType != null ? applicationType : "Apache Flink"); + appContext.setAMContainerSpec(amContainer); + appContext.setResource(capability); + + // Set priority for application + int priorityNum = flinkConfiguration.getInteger(YarnConfigOptions.APPLICATION_PRIORITY); + if (priorityNum >= 0) { + Priority priority = Priority.newInstance(priorityNum); + appContext.setPriority(priority); + } + + if (yarnQueue != null) { + appContext.setQueue(yarnQueue); + } + + setApplicationNodeLabel(appContext); + + setApplicationTags(appContext); + + // add a hook to clean up in case deployment fails + Thread deploymentFailureHook = new DeploymentFailureHook(yarnClient, yarnApplication, yarnFilesDir); + Runtime.getRuntime().addShutdownHook(deploymentFailureHook); + LOG.info("Submitting application master " + appId); + yarnClient.submitApplication(appContext); + + LOG.info("Waiting for the cluster to be allocated"); + final long startTime = System.currentTimeMillis(); + ApplicationReport report; + YarnApplicationState lastAppState = YarnApplicationState.NEW; + loop: while (true) { + try { + report = yarnClient.getApplicationReport(appId); + } catch (IOException e) { + throw new YarnDeploymentException("Failed to deploy the cluster.", e); + } + YarnApplicationState appState = report.getYarnApplicationState(); + LOG.debug("Application State: {}", appState); + switch(appState) { + case FAILED: + case FINISHED: + case KILLED: + throw new YarnDeploymentException("The YARN application unexpectedly switched to state " + + appState + " during deployment. \n" + + "Diagnostics from YARN: " + report.getDiagnostics() + "\n" + + "If log aggregation is enabled on your cluster, use this command to further investigate the issue:\n" + + "yarn logs -applicationId " + appId); + //break .. + case RUNNING: + LOG.info("YARN application has been deployed successfully."); + break loop; + default: + if (appState != lastAppState) { + LOG.info("Deploying cluster, current state " + appState); + } + if (System.currentTimeMillis() - startTime > 60000) { + LOG.info("Deployment took more than 60 seconds. Please check if the requested resources are available in the YARN cluster"); + } + + } + lastAppState = appState; + Thread.sleep(250); + } + // print the application id for user to cancel themselves. + if (isDetachedMode()) { + LOG.info("The Flink YARN client has been started in detached mode. In order to stop " + + "Flink on YARN, use the following command or a YARN web interface to stop " + + "it:\nyarn application -kill " + appId + "\nPlease also note that the " + + "temporary files of the YARN session in the home directory will not be removed."); + } + // since deployment was successful, remove the hook + ShutdownHookUtil.removeShutdownHook(deploymentFailureHook, getClass().getSimpleName(), LOG); + return report; + } + + /** + * Returns the Path where the YARN application files should be uploaded to. + * + * @param appId YARN application id + */ + private Path getYarnFilesDir(final ApplicationId appId) throws IOException { + final FileSystem fileSystem = FileSystem.get(yarnConfiguration); + final Path homeDir = fileSystem.getHomeDirectory(); + return new Path(homeDir, ".flink/" + appId + '/'); + } + + /** + * Uploads and registers a single resource and adds it to localResources. + * + * @param key + * the key to add the resource under + * @param fs + * the remote file system to upload to + * @param appId + * application ID + * @param localSrcPath + * local path to the file + * @param localResources + * map of resources + * + * @return the remote path to the uploaded resource + */ + private static Path setupSingleLocalResource( + String key, + FileSystem fs, + ApplicationId appId, + Path localSrcPath, + Map localResources, + Path targetHomeDir, + String relativeTargetPath) throws IOException { + Tuple2 resource = Utils.setupLocalResource( + fs, + appId.toString(), + localSrcPath, + targetHomeDir, + relativeTargetPath); + + localResources.put(key, resource.f1); + + return resource.f0; + } + + /** + * Match file name for "flink-dist*.jar" pattern. + * + * @param fileName file name to check + * @return true if file is a dist jar + */ + private static boolean isDistJar(String fileName) { + return fileName.startsWith("flink-dist") && fileName.endsWith("jar"); + } + + /** + * Recursively uploads (and registers) any (user and system) files in shipFiles except + * for files matching "flink-dist*.jar" which should be uploaded separately. + * + * @param shipFiles + * files to upload + * @param fs + * file system to upload to + * @param targetHomeDir + * remote home directory to upload to + * @param appId + * application ID + * @param remotePaths + * paths of the remote resources (uploaded resources will be added) + * @param localResources + * map of resources (uploaded resources will be added) + * @param envShipFileList + * list of shipped files in a format understood by {@link Utils#createTaskExecutorContext} + * + * @return list of class paths with the the proper resource keys from the registration + */ + static List uploadAndRegisterFiles( + Collection shipFiles, + FileSystem fs, + Path targetHomeDir, + ApplicationId appId, + List remotePaths, + Map localResources, + StringBuilder envShipFileList) throws IOException { + final List localPaths = new ArrayList<>(); + final List relativePaths = new ArrayList<>(); + for (File shipFile : shipFiles) { + if (shipFile.isDirectory()) { + // add directories to the classpath + final java.nio.file.Path shipPath = shipFile.toPath(); + final java.nio.file.Path parentPath = shipPath.getParent(); + Files.walkFileTree(shipPath, new SimpleFileVisitor() { + @Override + public FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes attrs) { + localPaths.add(new Path(file.toUri())); + relativePaths.add(new Path(parentPath.relativize(file).toString())); + return FileVisitResult.CONTINUE; + } + }); + } else { + localPaths.add(new Path(shipFile.toURI())); + relativePaths.add(new Path(shipFile.getName())); + } + } + + final Set archives = new HashSet<>(); + final Set resources = new HashSet<>(); + for (int i = 0; i < localPaths.size(); i++) { + final Path localPath = localPaths.get(i); + final Path relativePath = relativePaths.get(i); + if (!isDistJar(relativePath.getName())) { + final String key = relativePath.toString(); + final Path remotePath = setupSingleLocalResource( + key, + fs, + appId, + localPath, + localResources, + targetHomeDir, + relativePath.getParent().toString()); + remotePaths.add(remotePath); + envShipFileList.append(key).append("=").append(remotePath).append(","); + // add files to the classpath + if (key.endsWith("jar")) { + archives.add(relativePath.toString()); + } else { + resources.add(relativePath.getParent().toString()); + } + } + } + + // construct classpath, we always want resource directories to go first, we also sort + // both resources and archives in order to make classpath deterministic + final ArrayList classPaths = new ArrayList<>(); + resources.stream().sorted().forEach(classPaths::add); + archives.stream().sorted().forEach(classPaths::add); + return classPaths; + } + + /** + * Kills YARN application and stops YARN client. + * + *

Use this method to kill the App before it has been properly deployed + */ + private void failSessionDuringDeployment(YarnClient yarnClient, YarnClientApplication yarnApplication) { + LOG.info("Killing YARN application"); + + try { + yarnClient.killApplication(yarnApplication.getNewApplicationResponse().getApplicationId()); + } catch (Exception e) { + // we only log a debug message here because the "killApplication" call is a best-effort + // call (we don't know if the application has been deployed when the error occured). + LOG.debug("Error while killing YARN application", e); + } + yarnClient.stop(); + } + + private static class ClusterResourceDescription { + public final int totalFreeMemory; + public final int containerLimit; + public final int[] nodeManagersFree; + + public ClusterResourceDescription(int totalFreeMemory, int containerLimit, int[] nodeManagersFree) { + this.totalFreeMemory = totalFreeMemory; + this.containerLimit = containerLimit; + this.nodeManagersFree = nodeManagersFree; + } + } + + private ClusterResourceDescription getCurrentFreeClusterResources(YarnClient yarnClient) throws YarnException, IOException { + List nodes = yarnClient.getNodeReports(NodeState.RUNNING); + + int totalFreeMemory = 0; + int containerLimit = 0; + int[] nodeManagersFree = new int[nodes.size()]; + + for (int i = 0; i < nodes.size(); i++) { + NodeReport rep = nodes.get(i); + int free = rep.getCapability().getMemory() - (rep.getUsed() != null ? rep.getUsed().getMemory() : 0); + nodeManagersFree[i] = free; + totalFreeMemory += free; + if (free > containerLimit) { + containerLimit = free; + } + } + return new ClusterResourceDescription(totalFreeMemory, containerLimit, nodeManagersFree); + } + + @Override + public String getClusterDescription() { + + try { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PrintStream ps = new PrintStream(baos); + + YarnClusterMetrics metrics = yarnClient.getYarnClusterMetrics(); + + ps.append("NodeManagers in the ClusterClient " + metrics.getNumNodeManagers()); + List nodes = yarnClient.getNodeReports(NodeState.RUNNING); + final String format = "|%-16s |%-16s %n"; + ps.printf("|Property |Value %n"); + ps.println("+---------------------------------------+"); + int totalMemory = 0; + int totalCores = 0; + for (NodeReport rep : nodes) { + final Resource res = rep.getCapability(); + totalMemory += res.getMemory(); + totalCores += res.getVirtualCores(); + ps.format(format, "NodeID", rep.getNodeId()); + ps.format(format, "Memory", res.getMemory() + " MB"); + ps.format(format, "vCores", res.getVirtualCores()); + ps.format(format, "HealthReport", rep.getHealthReport()); + ps.format(format, "Containers", rep.getNumContainers()); + ps.println("+---------------------------------------+"); + } + ps.println("Summary: totalMemory " + totalMemory + " totalCores " + totalCores); + List qInfo = yarnClient.getAllQueues(); + for (QueueInfo q : qInfo) { + ps.println("Queue: " + q.getQueueName() + ", Current Capacity: " + q.getCurrentCapacity() + " Max Capacity: " + + q.getMaximumCapacity() + " Applications: " + q.getApplications().size()); + } + return baos.toString(); + } catch (Exception e) { + throw new RuntimeException("Couldn't get cluster description", e); + } + } + + public void setName(String name) { + this.customName = Preconditions.checkNotNull(name, "The customized name must not be null"); + } + + public void setApplicationType(String type) { + this.applicationType = Preconditions.checkNotNull(type, "The customized application type must not be null"); + } + + private void activateHighAvailabilitySupport(ApplicationSubmissionContext appContext) throws + InvocationTargetException, IllegalAccessException { + + ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); + + reflector.setKeepContainersAcrossApplicationAttempts(appContext, true); + + reflector.setAttemptFailuresValidityInterval( + appContext, + flinkConfiguration.getLong(YarnConfigOptions.APPLICATION_ATTEMPT_FAILURE_VALIDITY_INTERVAL)); + } + + private void setApplicationTags(final ApplicationSubmissionContext appContext) throws InvocationTargetException, + IllegalAccessException { + + final ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); + final String tagsString = flinkConfiguration.getString(YarnConfigOptions.APPLICATION_TAGS); + + final Set applicationTags = new HashSet<>(); + + // Trim whitespace and cull empty tags + for (final String tag : tagsString.split(",")) { + final String trimmedTag = tag.trim(); + if (!trimmedTag.isEmpty()) { + applicationTags.add(trimmedTag); + } + } + + reflector.setApplicationTags(appContext, applicationTags); + } + + private void setApplicationNodeLabel(final ApplicationSubmissionContext appContext) throws InvocationTargetException, + IllegalAccessException { + + if (nodeLabel != null) { + final ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); + reflector.setApplicationNodeLabel(appContext, nodeLabel); + } + } + + /** + * Singleton object which uses reflection to determine whether the {@link ApplicationSubmissionContext} + * supports various methods which, depending on the Hadoop version, may or may not be supported. + * + *

If an unsupported method is invoked, nothing happens. + * + *

Currently three methods are proxied: + * - setApplicationTags (>= 2.4.0) + * - setAttemptFailuresValidityInterval (>= 2.6.0) + * - setKeepContainersAcrossApplicationAttempts (>= 2.4.0) + * - setNodeLabelExpression (>= 2.6.0) + */ + private static class ApplicationSubmissionContextReflector { + private static final Logger LOG = LoggerFactory.getLogger(ApplicationSubmissionContextReflector.class); + + private static final ApplicationSubmissionContextReflector instance = + new ApplicationSubmissionContextReflector(ApplicationSubmissionContext.class); + + public static ApplicationSubmissionContextReflector getInstance() { + return instance; + } + + private static final String APPLICATION_TAGS_METHOD_NAME = "setApplicationTags"; + private static final String ATTEMPT_FAILURES_METHOD_NAME = "setAttemptFailuresValidityInterval"; + private static final String KEEP_CONTAINERS_METHOD_NAME = "setKeepContainersAcrossApplicationAttempts"; + private static final String NODE_LABEL_EXPRESSION_NAME = "setNodeLabelExpression"; + + private final Method applicationTagsMethod; + private final Method attemptFailuresValidityIntervalMethod; + private final Method keepContainersMethod; + @Nullable + private final Method nodeLabelExpressionMethod; + + private ApplicationSubmissionContextReflector(Class clazz) { + Method applicationTagsMethod; + Method attemptFailuresValidityIntervalMethod; + Method keepContainersMethod; + Method nodeLabelExpressionMethod; + + try { + // this method is only supported by Hadoop 2.4.0 onwards + applicationTagsMethod = clazz.getMethod(APPLICATION_TAGS_METHOD_NAME, Set.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), APPLICATION_TAGS_METHOD_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), APPLICATION_TAGS_METHOD_NAME); + // assign null because the Hadoop version apparently does not support this call. + applicationTagsMethod = null; + } + + this.applicationTagsMethod = applicationTagsMethod; + + try { + // this method is only supported by Hadoop 2.6.0 onwards + attemptFailuresValidityIntervalMethod = clazz.getMethod(ATTEMPT_FAILURES_METHOD_NAME, long.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), ATTEMPT_FAILURES_METHOD_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), ATTEMPT_FAILURES_METHOD_NAME); + // assign null because the Hadoop version apparently does not support this call. + attemptFailuresValidityIntervalMethod = null; + } + + this.attemptFailuresValidityIntervalMethod = attemptFailuresValidityIntervalMethod; + + try { + // this method is only supported by Hadoop 2.4.0 onwards + keepContainersMethod = clazz.getMethod(KEEP_CONTAINERS_METHOD_NAME, boolean.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); + // assign null because the Hadoop version apparently does not support this call. + keepContainersMethod = null; + } + + this.keepContainersMethod = keepContainersMethod; + + try { + nodeLabelExpressionMethod = clazz.getMethod(NODE_LABEL_EXPRESSION_NAME, String.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), NODE_LABEL_EXPRESSION_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), NODE_LABEL_EXPRESSION_NAME); + nodeLabelExpressionMethod = null; + } + + this.nodeLabelExpressionMethod = nodeLabelExpressionMethod; + } + + public void setApplicationTags( + ApplicationSubmissionContext appContext, + Set applicationTags) throws InvocationTargetException, IllegalAccessException { + if (applicationTagsMethod != null) { + LOG.debug("Calling method {} of {}.", + applicationTagsMethod.getName(), + appContext.getClass().getCanonicalName()); + applicationTagsMethod.invoke(appContext, applicationTags); + } else { + LOG.debug("{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), + APPLICATION_TAGS_METHOD_NAME); + } + } + + public void setApplicationNodeLabel( + ApplicationSubmissionContext appContext, + String nodeLabel) throws InvocationTargetException, IllegalAccessException { + if (nodeLabelExpressionMethod != null) { + LOG.debug("Calling method {} of {}.", + nodeLabelExpressionMethod.getName(), + appContext.getClass().getCanonicalName()); + nodeLabelExpressionMethod.invoke(appContext, nodeLabel); + } else { + LOG.debug("{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), + NODE_LABEL_EXPRESSION_NAME); + } + } + + public void setAttemptFailuresValidityInterval( + ApplicationSubmissionContext appContext, + long validityInterval) throws InvocationTargetException, IllegalAccessException { + if (attemptFailuresValidityIntervalMethod != null) { + LOG.debug("Calling method {} of {}.", + attemptFailuresValidityIntervalMethod.getName(), + appContext.getClass().getCanonicalName()); + attemptFailuresValidityIntervalMethod.invoke(appContext, validityInterval); + } else { + LOG.debug("{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), + ATTEMPT_FAILURES_METHOD_NAME); + } + } + + public void setKeepContainersAcrossApplicationAttempts( + ApplicationSubmissionContext appContext, + boolean keepContainers) throws InvocationTargetException, IllegalAccessException { + + if (keepContainersMethod != null) { + LOG.debug("Calling method {} of {}.", keepContainersMethod.getName(), + appContext.getClass().getCanonicalName()); + keepContainersMethod.invoke(appContext, keepContainers); + } else { + LOG.debug("{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); + } + } + } + + private static class YarnDeploymentException extends RuntimeException { + private static final long serialVersionUID = -812040641215388943L; + + public YarnDeploymentException(String message) { + super(message); + } + + public YarnDeploymentException(String message, Throwable cause) { + super(message, cause); + } + } + + private class DeploymentFailureHook extends Thread { + + private final YarnClient yarnClient; + private final YarnClientApplication yarnApplication; + private final Path yarnFilesDir; + + DeploymentFailureHook(YarnClient yarnClient, YarnClientApplication yarnApplication, Path yarnFilesDir) { + this.yarnClient = Preconditions.checkNotNull(yarnClient); + this.yarnApplication = Preconditions.checkNotNull(yarnApplication); + this.yarnFilesDir = Preconditions.checkNotNull(yarnFilesDir); + } + + @Override + public void run() { + LOG.info("Cancelling deployment from Deployment Failure Hook"); + failSessionDuringDeployment(yarnClient, yarnApplication); + LOG.info("Deleting files in {}.", yarnFilesDir); + try { + FileSystem fs = FileSystem.get(yarnConfiguration); + + if (!fs.delete(yarnFilesDir, true)) { + throw new IOException("Deleting files in " + yarnFilesDir + " was unsuccessful"); + } + + fs.close(); + } catch (IOException e) { + LOG.error("Failed to delete Flink Jar and configuration files in HDFS", e); + } + } + } + + protected void addEnvironmentFoldersToShipFiles(Collection effectiveShipFiles) { + addLibFoldersToShipFiles(effectiveShipFiles); + addPluginsFoldersToShipFiles(effectiveShipFiles); + } + + private void addLibFoldersToShipFiles(Collection effectiveShipFiles) { + // Add lib folder to the ship files if the environment variable is set. + // This is for convenience when running from the command-line. + // (for other files users explicitly set the ship files) + String libDir = System.getenv().get(ENV_FLINK_LIB_DIR); + if (libDir != null) { + File directoryFile = new File(libDir); + if (directoryFile.isDirectory()) { + effectiveShipFiles.add(directoryFile); + } else { + throw new YarnDeploymentException("The environment variable '" + ENV_FLINK_LIB_DIR + + "' is set to '" + libDir + "' but the directory doesn't exist."); + } + } else if (this.shipFiles.isEmpty()) { + LOG.warn("Environment variable '{}' not set and ship files have not been provided manually. " + + "Not shipping any library files.", ENV_FLINK_LIB_DIR); + } + } + + private void addPluginsFoldersToShipFiles(Collection effectiveShipFiles) { + String pluginsDir = System.getenv().get(ENV_FLINK_PLUGINS_DIR); + if (pluginsDir != null) { + File directoryFile = new File(pluginsDir); + if (directoryFile.isDirectory()) { + effectiveShipFiles.add(directoryFile); + } else { + LOG.warn("The environment variable '" + ENV_FLINK_PLUGINS_DIR + + "' is set to '" + pluginsDir + "' but the directory doesn't exist."); + } + } + } + + protected ContainerLaunchContext setupApplicationMasterContainer( + String yarnClusterEntrypoint, + boolean hasLogback, + boolean hasLog4j, + boolean hasKrb5, + int jobManagerMemoryMb) { + // ------------------ Prepare Application Master Container ------------------------------ + + // respect custom JVM options in the YAML file + String javaOpts = flinkConfiguration.getString(CoreOptions.FLINK_JVM_OPTIONS); + if (flinkConfiguration.getString(CoreOptions.FLINK_JM_JVM_OPTIONS).length() > 0) { + javaOpts += " " + flinkConfiguration.getString(CoreOptions.FLINK_JM_JVM_OPTIONS); + } + //applicable only for YarnMiniCluster secure test run + //krb5.conf file will be available as local resource in JM/TM container + if (hasKrb5) { + javaOpts += " -Djava.security.krb5.conf=krb5.conf"; + } + + // Set up the container launch context for the application master + ContainerLaunchContext amContainer = Records.newRecord(ContainerLaunchContext.class); + + final Map startCommandValues = new HashMap<>(); + startCommandValues.put("java", "$JAVA_HOME/bin/java"); + + int heapSize = Utils.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration); + String jvmHeapMem = String.format("-Xms%sm -Xmx%sm", heapSize, heapSize); + startCommandValues.put("jvmmem", jvmHeapMem); + + startCommandValues.put("jvmopts", javaOpts); + String logging = ""; + + if (hasLogback || hasLog4j) { + logging = "-Dlog.file=\"" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.log\""; + + if (hasLogback) { + logging += " -Dlogback.configurationFile=file:" + CONFIG_FILE_LOGBACK_NAME; + } + + if (hasLog4j) { + logging += " -Dlog4j.configuration=file:" + CONFIG_FILE_LOG4J_NAME; + } + } + + startCommandValues.put("logging", logging); + startCommandValues.put("class", yarnClusterEntrypoint); + startCommandValues.put("redirects", + "1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.out " + + "2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.err"); + startCommandValues.put("args", ""); + + final String commandTemplate = flinkConfiguration + .getString(ConfigConstants.YARN_CONTAINER_START_COMMAND_TEMPLATE, + ConfigConstants.DEFAULT_YARN_CONTAINER_START_COMMAND_TEMPLATE); + final String amCommand = + BootstrapTools.getStartCommand(commandTemplate, startCommandValues); + + amContainer.setCommands(Collections.singletonList(amCommand)); + + LOG.debug("Application Master start command: " + amCommand); + + return amContainer; + } + + private static YarnConfigOptions.UserJarInclusion getUserJarInclusionMode(org.apache.flink.configuration.Configuration config) { + throwIfUserTriesToDisableUserJarInclusionInSystemClassPath(config); + + return config.getEnum(YarnConfigOptions.UserJarInclusion.class, YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR); + } + + private static void throwIfUserTriesToDisableUserJarInclusionInSystemClassPath(final Configuration config) { + final String userJarInclusion = config.getString(YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR); + if ("DISABLED".equalsIgnoreCase(userJarInclusion)) { + throw new IllegalArgumentException(String.format("Config option %s cannot be set to DISABLED anymore (see FLINK-11781)", + YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR.key())); + } + } + + /** + * Creates a YarnClusterClient; may be overridden in tests. + */ protected ClusterClient createYarnClusterClient( - AbstractYarnClusterDescriptor descriptor, - int numberTaskManagers, - int slotsPerTaskManager, - ApplicationReport report, - Configuration flinkConfiguration, - boolean perJobCluster) throws Exception { + YarnClusterDescriptor descriptor, + int numberTaskManagers, + int slotsPerTaskManager, + ApplicationReport report, + Configuration flinkConfiguration, + boolean perJobCluster) throws Exception { return new RestClusterClient<>( flinkConfiguration, report.getApplicationId()); } } + diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index 06ff77fa32..e87927897a 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -39,7 +39,6 @@ import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.ShutdownHookUtil; -import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.flink.yarn.configuration.YarnConfigOptions; @@ -263,13 +262,13 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine createYarnClusterClient(AbstractYarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, ApplicationReport report, Configuration flinkConfiguration, boolean perJobCluster) throws Exception { - throw new UnsupportedOperationException("Not needed for testing"); - } - - @Override - public ClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph, boolean detached) throws ClusterDeploymentException { - throw new UnsupportedOperationException("Not needed for testing"); - } - } } diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java index a71f139b58..6b9779b979 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java @@ -85,7 +85,7 @@ public class FlinkYarnSessionCliTest extends TestLogger { CommandLine cmd = parser.parse(options, new String[]{"run", "-j", "fake.jar", "-D", "akka.ask.timeout=5 min", "-D", "env.java.opts=-DappName=foobar", "-D", "security.ssl.internal.key-password=changeit"}); - AbstractYarnClusterDescriptor flinkYarnDescriptor = cli.createClusterDescriptor(cmd); + YarnClusterDescriptor flinkYarnDescriptor = cli.createClusterDescriptor(cmd); Assert.assertNotNull(flinkYarnDescriptor); @@ -110,7 +110,7 @@ public class FlinkYarnSessionCliTest extends TestLogger { final CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true); - AbstractYarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine); + YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine); final ClusterSpecification clusterSpecification = yarnCLI.getClusterSpecification(commandLine); @@ -132,7 +132,7 @@ public class FlinkYarnSessionCliTest extends TestLogger { final CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true); - AbstractYarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine); + YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine); // each task manager has 3 slots but the parallelism is 7. Thus the slots should be increased. assertTrue(descriptor.isDetachedMode()); @@ -152,7 +152,7 @@ public class FlinkYarnSessionCliTest extends TestLogger { CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true); - AbstractYarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine); + YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine); assertEquals(zkNamespaceCliInput, descriptor.getZookeeperNamespace()); } @@ -171,7 +171,7 @@ public class FlinkYarnSessionCliTest extends TestLogger { CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true); - AbstractYarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine); + YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine); assertEquals(nodeLabelCliInput, descriptor.getNodeLabel()); } @@ -246,7 +246,7 @@ public class FlinkYarnSessionCliTest extends TestLogger { final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()}, true); - final AbstractYarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine); + final YarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine); final Configuration clusterDescriptorConfiguration = clusterDescriptor.getFlinkConfiguration(); @@ -267,7 +267,7 @@ public class FlinkYarnSessionCliTest extends TestLogger { final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString(), "-yz", overrideZkNamespace}, true); - final AbstractYarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine); + final YarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine); final Configuration clusterDescriptorConfiguration = clusterDescriptor.getFlinkConfiguration(); @@ -462,7 +462,7 @@ public class FlinkYarnSessionCliTest extends TestLogger { final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); - AbstractYarnClusterDescriptor flinkYarnDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine); + YarnClusterDescriptor flinkYarnDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine); assertEquals(2, flinkYarnDescriptor.shipFiles.size()); diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java index d0fad6dd4c..13d296dfeb 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java @@ -179,7 +179,7 @@ public class YarnFileStageTest extends TestLogger { try { List remotePaths = new ArrayList<>(); HashMap localResources = new HashMap<>(); - final List classpath = AbstractYarnClusterDescriptor.uploadAndRegisterFiles( + final List classpath = YarnClusterDescriptor.uploadAndRegisterFiles( Collections.singletonList(new File(srcPath.toUri().getPath())), targetFileSystem, targetDir, -- Gitee From e3839aa3120c0816f29d7e2629dfd15ab6e710cd Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 17 Sep 2019 19:31:51 +0800 Subject: [PATCH 088/268] [FLINK-14050][yarn] Narrow method visibility in YarnClusterDescriptor --- .../flink/yarn/YarnClusterDescriptor.java | 26 ++++++++++++------- .../flink/yarn/FlinkYarnSessionCliTest.java | 2 +- .../flink/yarn/YarnClusterDescriptorTest.java | 18 ++++++------- 3 files changed, 26 insertions(+), 20 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index b1782416fa..805bdc942e 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.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.client.deployment.ClusterDeploymentException; import org.apache.flink.client.deployment.ClusterDescriptor; @@ -123,6 +124,9 @@ public class YarnClusterDescriptor implements ClusterDescriptor { /** True if the descriptor must not shut down the YarnClient. */ private final boolean sharedYarnClient; + /** Lazily initialized list of files to ship. */ + private final List shipFiles = new LinkedList<>(); + private String yarnQueue; private String configurationDirectory; @@ -131,9 +135,6 @@ public class YarnClusterDescriptor implements ClusterDescriptor { private String dynamicPropertiesEncoded; - /** Lazily initialized list of files to ship. */ - protected List shipFiles = new LinkedList<>(); - private final Configuration flinkConfiguration; private boolean detached; @@ -160,11 +161,16 @@ public class YarnClusterDescriptor implements ClusterDescriptor { this.sharedYarnClient = sharedYarnClient; this.flinkConfiguration = Preconditions.checkNotNull(flinkConfiguration); - userJarInclusion = getUserJarInclusionMode(flinkConfiguration); + this.userJarInclusion = getUserJarInclusionMode(flinkConfiguration); this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory); } + @VisibleForTesting + List getShipFiles() { + return shipFiles; + } + public YarnClient getYarnClient() { return yarnClient; } @@ -456,7 +462,7 @@ public class YarnClusterDescriptor implements ClusterDescriptor { * @param jobGraph A job graph which is deployed with the Flink cluster, {@code null} if none * @param detached True if the cluster should be started in detached mode */ - protected ClusterClient deployInternal( + private ClusterClient deployInternal( ClusterSpecification clusterSpecification, String applicationName, String yarnClusterEntrypoint, @@ -559,7 +565,7 @@ public class YarnClusterDescriptor implements ClusterDescriptor { true); } - protected ClusterSpecification validateClusterResources( + private ClusterSpecification validateClusterResources( ClusterSpecification clusterSpecification, int yarnMinAllocationMB, Resource maximumResourceCapability, @@ -674,7 +680,7 @@ public class YarnClusterDescriptor implements ClusterDescriptor { } } - public ApplicationReport startAppMaster( + private ApplicationReport startAppMaster( Configuration configuration, String applicationName, String yarnClusterEntrypoint, @@ -1551,7 +1557,7 @@ public class YarnClusterDescriptor implements ClusterDescriptor { } } - protected void addEnvironmentFoldersToShipFiles(Collection effectiveShipFiles) { + void addEnvironmentFoldersToShipFiles(Collection effectiveShipFiles) { addLibFoldersToShipFiles(effectiveShipFiles); addPluginsFoldersToShipFiles(effectiveShipFiles); } @@ -1569,7 +1575,7 @@ public class YarnClusterDescriptor implements ClusterDescriptor { throw new YarnDeploymentException("The environment variable '" + ENV_FLINK_LIB_DIR + "' is set to '" + libDir + "' but the directory doesn't exist."); } - } else if (this.shipFiles.isEmpty()) { + } else if (shipFiles.isEmpty()) { LOG.warn("Environment variable '{}' not set and ship files have not been provided manually. " + "Not shipping any library files.", ENV_FLINK_LIB_DIR); } @@ -1588,7 +1594,7 @@ public class YarnClusterDescriptor implements ClusterDescriptor { } } - protected ContainerLaunchContext setupApplicationMasterContainer( + ContainerLaunchContext setupApplicationMasterContainer( String yarnClusterEntrypoint, boolean hasLogback, boolean hasLog4j, diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java index 6b9779b979..2338875113 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java @@ -464,7 +464,7 @@ public class FlinkYarnSessionCliTest extends TestLogger { YarnClusterDescriptor flinkYarnDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine); - assertEquals(2, flinkYarnDescriptor.shipFiles.size()); + assertEquals(2, flinkYarnDescriptor.getShipFiles().size()); } diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java index 4940f12bd1..143749b049 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java @@ -426,8 +426,8 @@ public class YarnClusterDescriptorTest extends TestLogger { File libFile = temporaryFolder.newFile("libFile.jar"); File libFolder = temporaryFolder.newFolder().getAbsoluteFile(); - Assert.assertFalse(descriptor.shipFiles.contains(libFile)); - Assert.assertFalse(descriptor.shipFiles.contains(libFolder)); + Assert.assertFalse(descriptor.getShipFiles().contains(libFile)); + Assert.assertFalse(descriptor.getShipFiles().contains(libFolder)); List shipFiles = new ArrayList<>(); shipFiles.add(libFile); @@ -435,17 +435,17 @@ public class YarnClusterDescriptorTest extends TestLogger { descriptor.addShipFiles(shipFiles); - Assert.assertTrue(descriptor.shipFiles.contains(libFile)); - Assert.assertTrue(descriptor.shipFiles.contains(libFolder)); + Assert.assertTrue(descriptor.getShipFiles().contains(libFile)); + Assert.assertTrue(descriptor.getShipFiles().contains(libFolder)); // only execute part of the deployment to test for shipped files Set effectiveShipFiles = new HashSet<>(); descriptor.addEnvironmentFoldersToShipFiles(effectiveShipFiles); Assert.assertEquals(0, effectiveShipFiles.size()); - Assert.assertEquals(2, descriptor.shipFiles.size()); - Assert.assertTrue(descriptor.shipFiles.contains(libFile)); - Assert.assertTrue(descriptor.shipFiles.contains(libFolder)); + Assert.assertEquals(2, descriptor.getShipFiles().size()); + Assert.assertTrue(descriptor.getShipFiles().contains(libFile)); + Assert.assertTrue(descriptor.getShipFiles().contains(libFolder)); } } @@ -481,8 +481,8 @@ public class YarnClusterDescriptorTest extends TestLogger { // only add the ship the folder, not the contents Assert.assertFalse(effectiveShipFiles.contains(libFile)); Assert.assertTrue(effectiveShipFiles.contains(libFolder)); - Assert.assertFalse(descriptor.shipFiles.contains(libFile)); - Assert.assertFalse(descriptor.shipFiles.contains(libFolder)); + Assert.assertFalse(descriptor.getShipFiles().contains(libFile)); + Assert.assertFalse(descriptor.getShipFiles().contains(libFolder)); } } -- Gitee From 542751ce6af4ab6b507a7c808f975182ce94d560 Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 17 Sep 2019 19:34:33 +0800 Subject: [PATCH 089/268] [FLINK-14050][yarn] Remove unused parameter from #createYarnClusterClient --- .../NonDeployingYarnClusterDescriptor.java | 8 ++------ .../flink/yarn/YarnClusterDescriptor.java | 20 +++++-------------- 2 files changed, 7 insertions(+), 21 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/NonDeployingYarnClusterDescriptor.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/NonDeployingYarnClusterDescriptor.java index 0264a9f4ed..26719362c8 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/NonDeployingYarnClusterDescriptor.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/NonDeployingYarnClusterDescriptor.java @@ -55,12 +55,8 @@ public class NonDeployingYarnClusterDescriptor extends YarnClusterDescriptor { @Override protected ClusterClient createYarnClusterClient( - YarnClusterDescriptor descriptor, - int numberTaskManagers, - int slotsPerTaskManager, - ApplicationReport report, - Configuration flinkConfiguration, - boolean perJobCluster) { + ApplicationReport report, + Configuration flinkConfiguration) { return clusterClient; } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index 805bdc942e..6b7e3d9106 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -373,12 +373,9 @@ public class YarnClusterDescriptor implements ClusterDescriptor { flinkConfiguration.setInteger(RestOptions.PORT, port); return createYarnClusterClient( - this, - -1, // we don't know the number of task managers of a started Flink cluster - -1, // we don't know how many slots each task manager has for a started Flink cluster report, - flinkConfiguration, - false); + flinkConfiguration + ); } catch (Exception e) { throw new ClusterRetrieveException("Couldn't retrieve Yarn cluster", e); } @@ -557,12 +554,9 @@ public class YarnClusterDescriptor implements ClusterDescriptor { // the Flink cluster is deployed in YARN. Represent cluster return createYarnClusterClient( - this, - validClusterSpecification.getNumberTaskManagers(), - validClusterSpecification.getSlotsPerTaskManager(), report, - flinkConfiguration, - true); + flinkConfiguration + ); } private ClusterSpecification validateClusterResources( @@ -1676,12 +1670,8 @@ public class YarnClusterDescriptor implements ClusterDescriptor { * Creates a YarnClusterClient; may be overridden in tests. */ protected ClusterClient createYarnClusterClient( - YarnClusterDescriptor descriptor, - int numberTaskManagers, - int slotsPerTaskManager, ApplicationReport report, - Configuration flinkConfiguration, - boolean perJobCluster) throws Exception { + Configuration flinkConfiguration) throws Exception { return new RestClusterClient<>( flinkConfiguration, report.getApplicationId()); -- Gitee From 60488e7b93b65beda4db6d2ee14fa171a1f87a06 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 25 Sep 2019 11:20:20 +0200 Subject: [PATCH 090/268] [FLINK-14205] Return DuplicateJobSubmissionException if submitted job is a duplicate Dispatcher#submitJob now returns a DuplicateJobSubmissionException, which is a subclass of JobSubmissionException, if the submitted job is a duplicate. This allows to better distinguish duplicate job submissions from other job submission errors. This closes #9771. --- .../DuplicateJobSubmissionException.java | 32 +++++++++++++++++++ .../flink/runtime/dispatcher/Dispatcher.java | 3 +- .../DispatcherResourceCleanupTest.java | 5 +-- 3 files changed, 37 insertions(+), 3 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/client/DuplicateJobSubmissionException.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/DuplicateJobSubmissionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/DuplicateJobSubmissionException.java new file mode 100644 index 0000000000..6399aebabb --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/DuplicateJobSubmissionException.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.runtime.client; + +import org.apache.flink.api.common.JobID; + +/** + * Exception which is returned upon job submission if the submitted job + * is currently being executed. + */ +public class DuplicateJobSubmissionException extends JobSubmissionException { + + public DuplicateJobSubmissionException(JobID jobID) { + super(jobID, "Job has already been submitted."); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index c4beb01528..7b72962e40 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -27,6 +27,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.checkpoint.Checkpoints; +import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; @@ -265,7 +266,7 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme try { if (isDuplicateJob(jobGraph.getJobID())) { return FutureUtils.completedExceptionally( - new JobSubmissionException(jobGraph.getJobID(), "Job has already been submitted.")); + new DuplicateJobSubmissionException(jobGraph.getJobID())); } else if (isPartialResourceConfigured(jobGraph)) { return FutureUtils.completedExceptionally( new JobSubmissionException(jobGraph.getJobID(), "Currently jobs is not supported if parts of the vertices have " + diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 182f11d8f5..313db2a844 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -29,6 +29,7 @@ import org.apache.flink.runtime.blob.BlobStore; import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.blob.TestingBlobStore; import org.apache.flink.runtime.blob.TestingBlobStoreBuilder; +import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; @@ -370,9 +371,9 @@ public class DispatcherResourceCleanupTest extends TestLogger { try { try { submissionFuture.get(); - fail("Expected a JobSubmissionFailure."); + fail("Expected a DuplicateJobSubmissionFailure."); } catch (ExecutionException ee) { - assertThat(ExceptionUtils.findThrowable(ee, JobSubmissionException.class).isPresent(), is(true)); + assertThat(ExceptionUtils.findThrowable(ee, DuplicateJobSubmissionException.class).isPresent(), is(true)); } assertThatHABlobsHaveNotBeenRemoved(); -- Gitee From 81038ff1ec0f809f19677a469447053d489f964c Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Mon, 30 Sep 2019 11:26:37 +0800 Subject: [PATCH 091/268] [FLINK-14178][build] Downgrade maven-shade-plugin to 3.1.1 This closes #9817. --- docs/dev/projectsetup/dependencies.md | 2 +- docs/dev/projectsetup/dependencies.zh.md | 2 +- pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/dev/projectsetup/dependencies.md b/docs/dev/projectsetup/dependencies.md index 258a72b793..ce71f51d52 100644 --- a/docs/dev/projectsetup/dependencies.md +++ b/docs/dev/projectsetup/dependencies.md @@ -196,7 +196,7 @@ you can use the following shade plugin definition: org.apache.maven.plugins maven-shade-plugin - 3.2.1 + 3.1.1 package diff --git a/docs/dev/projectsetup/dependencies.zh.md b/docs/dev/projectsetup/dependencies.zh.md index ccf7af2875..73a2e5c89e 100644 --- a/docs/dev/projectsetup/dependencies.zh.md +++ b/docs/dev/projectsetup/dependencies.zh.md @@ -159,7 +159,7 @@ Scala 版本(2.10、2.11、2.12等)互相是不兼容的。因此,依赖 Scala org.apache.maven.plugins maven-shade-plugin - 3.2.1 + 3.1.1 package diff --git a/pom.xml b/pom.xml index 832e298d85..73ac0aebbe 100644 --- a/pom.xml +++ b/pom.xml @@ -1686,7 +1686,7 @@ under the License. org.apache.maven.plugins maven-shade-plugin - 3.2.1 + 3.1.1 -- Gitee From a3890103d2fb1319d289b83d2451136751ca59c4 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 6 May 2019 12:43:50 +0200 Subject: [PATCH 092/268] [hotfix] Remove Exception from ZooKeeperUtils#createLeaderRetrievalService --- .../java/org/apache/flink/runtime/util/ZooKeeperUtils.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java index 084d021c9c..f29f3351cd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java @@ -172,11 +172,10 @@ public class ZooKeeperUtils { * @param client The {@link CuratorFramework} ZooKeeper client to use * @param configuration {@link Configuration} object containing the configuration values * @return {@link ZooKeeperLeaderRetrievalService} instance. - * @throws Exception */ public static ZooKeeperLeaderRetrievalService createLeaderRetrievalService( final CuratorFramework client, - final Configuration configuration) throws Exception { + final Configuration configuration) { return createLeaderRetrievalService(client, configuration, ""); } -- Gitee From 257ba6e4c0fa72b94dc0427d00c39921cc57d129 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 3 May 2019 18:07:48 +0200 Subject: [PATCH 093/268] [FLINK-14251] Add FutureUtils#forward utility The forward function completes the second future with the result of the first future. This closes #9786. --- .../flink/runtime/concurrent/FutureUtils.java | 17 +++++++ .../runtime/concurrent/FutureUtilsTest.java | 45 +++++++++++++++++++ 2 files changed, 62 insertions(+) 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 b9e5ed463e..73f3ecb663 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 @@ -1066,4 +1066,21 @@ public class FutureUtils { throw suppressedExceptions; } } + + /** + * Forwards the value from the source future to the target future. + * + * @param source future to forward the value from + * @param target future to forward the value to + * @param type of the value + */ + public static void forward(CompletableFuture source, CompletableFuture target) { + source.whenComplete((value, throwable) -> { + if (throwable != null) { + target.completeExceptionally(throwable); + } else { + target.complete(value); + } + }); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java index 95ee97a9b2..e75abb938b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java @@ -48,6 +48,7 @@ import java.util.function.BiFunction; import java.util.function.Supplier; import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.emptyArray; @@ -765,4 +766,48 @@ public class FutureUtilsTest extends TestLogger { return exception != null; } } + + @Test + public void testForwardNormal() throws Exception { + final CompletableFuture source = new CompletableFuture<>(); + final CompletableFuture target = new CompletableFuture<>(); + + FutureUtils.forward(source, target); + + assertThat(target.isDone(), is(source.isDone())); + + source.complete("foobar"); + + assertThat(target.isDone(), is(source.isDone())); + assertThat(target.get(), is(equalTo(source.get()))); + } + + @Test + public void testForwardExceptionally() { + final CompletableFuture source = new CompletableFuture<>(); + final CompletableFuture target = new CompletableFuture<>(); + + FutureUtils.forward(source, target); + + assertThat(target.isDone(), is(source.isDone())); + + source.completeExceptionally(new FlinkException("foobar")); + + assertThat(target.isDone(), is(source.isDone())); + + Throwable targetException = getThrowable(target); + Throwable actualException = getThrowable(source); + + assertThat(targetException, is(equalTo(actualException))); + } + + private static Throwable getThrowable(CompletableFuture completableFuture) { + try { + completableFuture.join(); + } catch (CompletionException e) { + return e.getCause(); + } + + throw new AssertionError("Future has not been completed exceptionally."); + } } -- Gitee From 470558aeb0b30587ac308f2e93bbe42131e03e42 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 29 Apr 2019 12:06:17 +0200 Subject: [PATCH 094/268] [FLINK-14252] Encapsulate Dispatcher services in DispatcherServices The DispatcherServices container contains all required Dispatcher services except for the JobGraphStore which needs to passed in separately to make it work with the MiniDispatcher. This closes #9787. --- .../flink/runtime/dispatcher/Dispatcher.java | 41 ++--- .../runtime/dispatcher/DispatcherFactory.java | 23 +-- .../dispatcher/DispatcherServices.java | 86 +++++++++++ .../dispatcher/JobDispatcherFactory.java | 35 +---- .../runtime/dispatcher/MiniDispatcher.java | 36 +---- .../dispatcher/PartialDispatcherServices.java | 141 ++++++++++++++++++ .../dispatcher/SessionDispatcherFactory.java | 35 +---- .../dispatcher/StandaloneDispatcher.java | 36 +---- ...atcherResourceManagerComponentFactory.java | 12 +- .../dispatcher/MiniDispatcherTest.java | 23 +-- .../runtime/dispatcher/TestingDispatcher.java | 25 ++-- .../SessionDispatcherWithUUIDFactory.java | 39 +---- 12 files changed, 300 insertions(+), 232 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServices.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 7b72962e40..96f1e912d1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -136,33 +136,24 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme public Dispatcher( RpcService rpcService, String endpointId, - Configuration configuration, - HighAvailabilityServices highAvailabilityServices, - JobGraphStore jobGraphStore, - GatewayRetriever resourceManagerGatewayRetriever, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - JobManagerMetricGroup jobManagerMetricGroup, - @Nullable String metricServiceQueryAddress, - ArchivedExecutionGraphStore archivedExecutionGraphStore, - JobManagerRunnerFactory jobManagerRunnerFactory, - FatalErrorHandler fatalErrorHandler, - HistoryServerArchivist historyServerArchivist) throws Exception { + DispatcherServices dispatcherServices, + JobGraphStore jobGraphStore) throws Exception { super(rpcService, endpointId); - - this.configuration = Preconditions.checkNotNull(configuration); - this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityServices); - this.resourceManagerGatewayRetriever = Preconditions.checkNotNull(resourceManagerGatewayRetriever); - this.heartbeatServices = Preconditions.checkNotNull(heartbeatServices); - this.blobServer = Preconditions.checkNotNull(blobServer); - this.fatalErrorHandler = Preconditions.checkNotNull(fatalErrorHandler); + Preconditions.checkNotNull(dispatcherServices); + + this.configuration = dispatcherServices.getConfiguration(); + this.highAvailabilityServices = dispatcherServices.getHighAvailabilityServices(); + this.resourceManagerGatewayRetriever = dispatcherServices.getResourceManagerGatewayRetriever(); + this.heartbeatServices = dispatcherServices.getHeartbeatServices(); + this.blobServer = dispatcherServices.getBlobServer(); + this.fatalErrorHandler = dispatcherServices.getFatalErrorHandler(); this.jobGraphStore = Preconditions.checkNotNull(jobGraphStore); - this.jobManagerMetricGroup = Preconditions.checkNotNull(jobManagerMetricGroup); - this.metricServiceQueryAddress = metricServiceQueryAddress; + this.jobManagerMetricGroup = dispatcherServices.getJobManagerMetricGroup(); + this.metricServiceQueryAddress = dispatcherServices.getMetricQueryServiceAddress(); this.jobManagerSharedServices = JobManagerSharedServices.fromConfiguration( configuration, - this.blobServer); + blobServer); this.runningJobsRegistry = highAvailabilityServices.getRunningJobsRegistry(); @@ -170,11 +161,11 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme leaderElectionService = highAvailabilityServices.getDispatcherLeaderElectionService(); - this.historyServerArchivist = Preconditions.checkNotNull(historyServerArchivist); + this.historyServerArchivist = dispatcherServices.getHistoryServerArchivist(); - this.archivedExecutionGraphStore = Preconditions.checkNotNull(archivedExecutionGraphStore); + this.archivedExecutionGraphStore = dispatcherServices.getArchivedExecutionGraphStore(); - this.jobManagerRunnerFactory = Preconditions.checkNotNull(jobManagerRunnerFactory); + this.jobManagerRunnerFactory = dispatcherServices.getJobManagerRunnerFactory(); this.jobManagerTerminationFutures = new HashMap<>(2); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java index 4c421a09d0..4de454278d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java @@ -18,17 +18,9 @@ package org.apache.flink.runtime.dispatcher; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; -import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import javax.annotation.Nullable; +import javax.annotation.Nonnull; import java.util.UUID; @@ -41,17 +33,8 @@ public interface DispatcherFactory { * Create a {@link Dispatcher} of the given type {@link T}. */ T createDispatcher( - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - GatewayRetriever resourceManagerGatewayRetriever, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - JobManagerMetricGroup jobManagerMetricGroup, - @Nullable String metricQueryServiceAddress, - ArchivedExecutionGraphStore archivedExecutionGraphStore, - FatalErrorHandler fatalErrorHandler, - HistoryServerArchivist historyServerArchivist) throws Exception; + @Nonnull RpcService rpcService, + @Nonnull PartialDispatcherServices partialDispatcherServices) throws Exception; default String generateEndpointIdWithUUID() { return getEndpointId() + UUID.randomUUID(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java new file mode 100644 index 0000000000..0d7cc88014 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +/** + * {@link Dispatcher} services container. + */ +public class DispatcherServices extends PartialDispatcherServices { + + @Nonnull + private final JobManagerRunnerFactory jobManagerRunnerFactory; + + public DispatcherServices( + @Nonnull Configuration configuration, + @Nonnull HighAvailabilityServices highAvailabilityServices, + @Nonnull GatewayRetriever resourceManagerGatewayRetriever, + @Nonnull BlobServer blobServer, + @Nonnull HeartbeatServices heartbeatServices, + @Nonnull JobManagerMetricGroup jobManagerMetricGroup, + @Nonnull ArchivedExecutionGraphStore archivedExecutionGraphStore, + @Nonnull FatalErrorHandler fatalErrorHandler, + @Nonnull HistoryServerArchivist historyServerArchivist, + @Nullable String metricQueryServiceAddress, + @Nonnull JobManagerRunnerFactory jobManagerRunnerFactory) { + super( + configuration, + highAvailabilityServices, + resourceManagerGatewayRetriever, + blobServer, + heartbeatServices, + jobManagerMetricGroup, + archivedExecutionGraphStore, + fatalErrorHandler, + historyServerArchivist, + metricQueryServiceAddress); + this.jobManagerRunnerFactory = jobManagerRunnerFactory; + } + + @Nonnull + JobManagerRunnerFactory getJobManagerRunnerFactory() { + return jobManagerRunnerFactory; + } + + public static DispatcherServices from(@Nonnull PartialDispatcherServices partialDispatcherServices, @Nonnull JobManagerRunnerFactory jobManagerRunnerFactory) { + return new DispatcherServices( + partialDispatcherServices.getConfiguration(), + partialDispatcherServices.getHighAvailabilityServices(), + partialDispatcherServices.getResourceManagerGatewayRetriever(), + partialDispatcherServices.getBlobServer(), + partialDispatcherServices.getHeartbeatServices(), + partialDispatcherServices.getJobManagerMetricGroup(), + partialDispatcherServices.getArchivedExecutionGraphStore(), + partialDispatcherServices.getFatalErrorHandler(), + partialDispatcherServices.getHistoryServerArchivist(), + partialDispatcherServices.getMetricQueryServiceAddress(), + jobManagerRunnerFactory); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java index f6c22c2fed..a74610e047 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java @@ -19,19 +19,12 @@ package org.apache.flink.runtime.dispatcher; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; -import org.apache.flink.runtime.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; -import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import javax.annotation.Nullable; +import javax.annotation.Nonnull; import static org.apache.flink.runtime.entrypoint.ClusterEntrypoint.EXECUTION_MODE; @@ -48,17 +41,9 @@ public class JobDispatcherFactory implements DispatcherFactory { @Override public MiniDispatcher createDispatcher( - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - GatewayRetriever resourceManagerGatewayRetriever, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - JobManagerMetricGroup jobManagerMetricGroup, - @Nullable String metricQueryServiceAddress, - ArchivedExecutionGraphStore archivedExecutionGraphStore, - FatalErrorHandler fatalErrorHandler, - HistoryServerArchivist historyServerArchivist) throws Exception { + @Nonnull RpcService rpcService, + @Nonnull PartialDispatcherServices partialDispatcherServices) throws Exception { + final Configuration configuration = partialDispatcherServices.getConfiguration(); final JobGraph jobGraph = jobGraphRetriever.retrieveJobGraph(configuration); final String executionModeValue = configuration.getString(EXECUTION_MODE); @@ -68,17 +53,7 @@ public class JobDispatcherFactory implements DispatcherFactory { return new MiniDispatcher( rpcService, getEndpointId(), - configuration, - highAvailabilityServices, - resourceManagerGatewayRetriever, - blobServer, - heartbeatServices, - jobManagerMetricGroup, - metricQueryServiceAddress, - archivedExecutionGraphStore, - DefaultJobManagerRunnerFactory.INSTANCE, - fatalErrorHandler, - historyServerArchivist, + DispatcherServices.from(partialDispatcherServices, DefaultJobManagerRunnerFactory.INSTANCE), jobGraph, executionMode); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java index eaea0b9235..6aefd00b0e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java @@ -20,26 +20,16 @@ package org.apache.flink.runtime.dispatcher; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.entrypoint.JobClusterEntrypoint; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; -import org.apache.flink.runtime.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; -import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.util.FlinkException; -import javax.annotation.Nullable; - import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -61,34 +51,14 @@ public class MiniDispatcher extends Dispatcher { public MiniDispatcher( RpcService rpcService, String endpointId, - Configuration configuration, - HighAvailabilityServices highAvailabilityServices, - GatewayRetriever resourceManagerGatewayRetriever, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - JobManagerMetricGroup jobManagerMetricGroup, - @Nullable String metricQueryServiceAddress, - ArchivedExecutionGraphStore archivedExecutionGraphStore, - JobManagerRunnerFactory jobManagerRunnerFactory, - FatalErrorHandler fatalErrorHandler, - HistoryServerArchivist historyServerArchivist, + DispatcherServices dispatcherServices, JobGraph jobGraph, JobClusterEntrypoint.ExecutionMode executionMode) throws Exception { super( rpcService, endpointId, - configuration, - highAvailabilityServices, - new SingleJobJobGraphStore(jobGraph), - resourceManagerGatewayRetriever, - blobServer, - heartbeatServices, - jobManagerMetricGroup, - metricQueryServiceAddress, - archivedExecutionGraphStore, - jobManagerRunnerFactory, - fatalErrorHandler, - historyServerArchivist); + dispatcherServices, + new SingleJobJobGraphStore(jobGraph)); this.executionMode = checkNotNull(executionMode); this.jobTerminationFuture = new CompletableFuture<>(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServices.java new file mode 100644 index 0000000000..3425ec7151 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServices.java @@ -0,0 +1,141 @@ +/* + * 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.dispatcher; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +/** + * Partial {@link DispatcherServices} services container which needs to + * be completed before being given to the {@link Dispatcher}. + */ +public class PartialDispatcherServices { + + @Nonnull + private final Configuration configuration; + + @Nonnull + private final HighAvailabilityServices highAvailabilityServices; + + @Nonnull + private final GatewayRetriever resourceManagerGatewayRetriever; + + @Nonnull + private final BlobServer blobServer; + + @Nonnull + private final HeartbeatServices heartbeatServices; + + @Nonnull + private final JobManagerMetricGroup jobManagerMetricGroup; + + @Nonnull + private final ArchivedExecutionGraphStore archivedExecutionGraphStore; + + @Nonnull + private final FatalErrorHandler fatalErrorHandler; + + @Nonnull + private final HistoryServerArchivist historyServerArchivist; + + @Nullable + private final String metricQueryServiceAddress; + + public PartialDispatcherServices( + @Nonnull Configuration configuration, + @Nonnull HighAvailabilityServices highAvailabilityServices, + @Nonnull GatewayRetriever resourceManagerGatewayRetriever, + @Nonnull BlobServer blobServer, + @Nonnull HeartbeatServices heartbeatServices, + @Nonnull JobManagerMetricGroup jobManagerMetricGroup, + @Nonnull ArchivedExecutionGraphStore archivedExecutionGraphStore, + @Nonnull FatalErrorHandler fatalErrorHandler, + @Nonnull HistoryServerArchivist historyServerArchivist, + @Nullable String metricQueryServiceAddress) { + this.configuration = configuration; + this.highAvailabilityServices = highAvailabilityServices; + this.resourceManagerGatewayRetriever = resourceManagerGatewayRetriever; + this.blobServer = blobServer; + this.heartbeatServices = heartbeatServices; + this.jobManagerMetricGroup = jobManagerMetricGroup; + this.archivedExecutionGraphStore = archivedExecutionGraphStore; + this.fatalErrorHandler = fatalErrorHandler; + this.historyServerArchivist = historyServerArchivist; + this.metricQueryServiceAddress = metricQueryServiceAddress; + } + + @Nonnull + public Configuration getConfiguration() { + return configuration; + } + + @Nonnull + public HighAvailabilityServices getHighAvailabilityServices() { + return highAvailabilityServices; + } + + @Nonnull + public GatewayRetriever getResourceManagerGatewayRetriever() { + return resourceManagerGatewayRetriever; + } + + @Nonnull + public BlobServer getBlobServer() { + return blobServer; + } + + @Nonnull + public HeartbeatServices getHeartbeatServices() { + return heartbeatServices; + } + + @Nonnull + public JobManagerMetricGroup getJobManagerMetricGroup() { + return jobManagerMetricGroup; + } + + @Nonnull + public ArchivedExecutionGraphStore getArchivedExecutionGraphStore() { + return archivedExecutionGraphStore; + } + + @Nonnull + public FatalErrorHandler getFatalErrorHandler() { + return fatalErrorHandler; + } + + @Nonnull + public HistoryServerArchivist getHistoryServerArchivist() { + return historyServerArchivist; + } + + @Nullable + public String getMetricQueryServiceAddress() { + return metricQueryServiceAddress; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java index 18fb0be937..f9b373c029 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java @@ -18,17 +18,9 @@ package org.apache.flink.runtime.dispatcher; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; -import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import javax.annotation.Nullable; +import javax.annotation.Nonnull; /** * {@link DispatcherFactory} which creates a {@link StandaloneDispatcher}. @@ -38,31 +30,12 @@ public enum SessionDispatcherFactory implements DispatcherFactory { @Override public Dispatcher createDispatcher( - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - GatewayRetriever resourceManagerGatewayRetriever, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - JobManagerMetricGroup jobManagerMetricGroup, - @Nullable String metricQueryServiceAddress, - ArchivedExecutionGraphStore archivedExecutionGraphStore, - FatalErrorHandler fatalErrorHandler, - HistoryServerArchivist historyServerArchivist) throws Exception { + @Nonnull RpcService rpcService, + @Nonnull PartialDispatcherServices partialDispatcherServices) throws Exception { // create the default dispatcher return new StandaloneDispatcher( rpcService, getEndpointId(), - configuration, - highAvailabilityServices, - resourceManagerGatewayRetriever, - blobServer, - heartbeatServices, - jobManagerMetricGroup, - metricQueryServiceAddress, - archivedExecutionGraphStore, - DefaultJobManagerRunnerFactory.INSTANCE, - fatalErrorHandler, - historyServerArchivist); + DispatcherServices.from(partialDispatcherServices, DefaultJobManagerRunnerFactory.INSTANCE)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java index bd14ca9f03..90b7faa647 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java @@ -18,19 +18,9 @@ package org.apache.flink.runtime.dispatcher; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobMaster; -import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; -import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; - -import javax.annotation.Nullable; /** * Dispatcher implementation which spawns a {@link JobMaster} for each @@ -41,31 +31,11 @@ public class StandaloneDispatcher extends Dispatcher { public StandaloneDispatcher( RpcService rpcService, String endpointId, - Configuration configuration, - HighAvailabilityServices highAvailabilityServices, - GatewayRetriever resourceManagerGatewayRetriever, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - JobManagerMetricGroup jobManagerMetricGroup, - @Nullable String metricQueryServiceAddress, - ArchivedExecutionGraphStore archivedExecutionGraphStore, - JobManagerRunnerFactory jobManagerRunnerFactory, - FatalErrorHandler fatalErrorHandler, - HistoryServerArchivist historyServerArchivist) throws Exception { + DispatcherServices dispatcherServices) throws Exception { super( rpcService, endpointId, - configuration, - highAvailabilityServices, - highAvailabilityServices.getJobGraphStore(), - resourceManagerGatewayRetriever, - blobServer, - heartbeatServices, - jobManagerMetricGroup, - metricQueryServiceAddress, - archivedExecutionGraphStore, - jobManagerRunnerFactory, - fatalErrorHandler, - historyServerArchivist); + dispatcherServices, + dispatcherServices.getHighAvailabilityServices().getJobGraphStore()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java index 4a51d7575b..6d50ed0490 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.dispatcher.HistoryServerArchivist; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -181,18 +182,21 @@ public abstract class AbstractDispatcherResourceManagerComponentFactory CompletableFuture.completedFuture(resourceManagerGateway), - blobServer, - heartbeatServices, - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), - null, - archivedExecutionGraphStore, - testingJobManagerRunnerFactory, - testingFatalErrorHandler, - VoidHistoryServerArchivist.INSTANCE, + new DispatcherServices( + configuration, + highAvailabilityServices, + () -> CompletableFuture.completedFuture(resourceManagerGateway), + blobServer, + heartbeatServices, + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), + archivedExecutionGraphStore, + testingFatalErrorHandler, + VoidHistoryServerArchivist.INSTANCE, + null, + testingJobManagerRunnerFactory), jobGraph, executionMode); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java index 43fc96f258..90b70532d8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java @@ -60,18 +60,19 @@ class TestingDispatcher extends Dispatcher { super( rpcService, endpointId, - configuration, - highAvailabilityServices, - highAvailabilityServices.getJobGraphStore(), - resourceManagerGatewayRetriever, - blobServer, - heartbeatServices, - jobManagerMetricGroup, - metricQueryServiceAddress, - archivedExecutionGraphStore, - jobManagerRunnerFactory, - fatalErrorHandler, - VoidHistoryServerArchivist.INSTANCE); + new DispatcherServices( + configuration, + highAvailabilityServices, + resourceManagerGatewayRetriever, + blobServer, + heartbeatServices, + jobManagerMetricGroup, + archivedExecutionGraphStore, + fatalErrorHandler, + VoidHistoryServerArchivist.INSTANCE, + metricQueryServiceAddress, + jobManagerRunnerFactory), + highAvailabilityServices.getJobGraphStore()); this.startFuture = new CompletableFuture<>(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/SessionDispatcherWithUUIDFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/SessionDispatcherWithUUIDFactory.java index e735dfe079..a85ec92790 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/SessionDispatcherWithUUIDFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/SessionDispatcherWithUUIDFactory.java @@ -18,23 +18,15 @@ package org.apache.flink.runtime.minicluster; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.dispatcher.ArchivedExecutionGraphStore; import org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherFactory; -import org.apache.flink.runtime.dispatcher.HistoryServerArchivist; +import org.apache.flink.runtime.dispatcher.DispatcherServices; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; -import org.apache.flink.runtime.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; -import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import javax.annotation.Nullable; +import javax.annotation.Nonnull; /** * {@link DispatcherFactory} which creates a {@link StandaloneDispatcher} which has an @@ -45,31 +37,12 @@ public enum SessionDispatcherWithUUIDFactory implements DispatcherFactory resourceManagerGatewayRetriever, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - JobManagerMetricGroup jobManagerMetricGroup, - @Nullable String metricQueryServiceAddress, - ArchivedExecutionGraphStore archivedExecutionGraphStore, - FatalErrorHandler fatalErrorHandler, - HistoryServerArchivist historyServerArchivist) throws Exception { + @Nonnull RpcService rpcService, + @Nonnull PartialDispatcherServices partialDispatcherServices) throws Exception { // create the default dispatcher return new StandaloneDispatcher( rpcService, generateEndpointIdWithUUID(), - configuration, - highAvailabilityServices, - resourceManagerGatewayRetriever, - blobServer, - heartbeatServices, - jobManagerMetricGroup, - metricQueryServiceAddress, - archivedExecutionGraphStore, - DefaultJobManagerRunnerFactory.INSTANCE, - fatalErrorHandler, - historyServerArchivist); + DispatcherServices.from(partialDispatcherServices, DefaultJobManagerRunnerFactory.INSTANCE)); } } -- Gitee From 1df5a0f4c63f321821c680dbab5b60c08ce37510 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 7 May 2019 11:01:22 +0200 Subject: [PATCH 095/268] [FLINK-14259] Introduce JobManagerRunner interface This commit extracts the JobManagerRunner interface and renames the implementation class into JobManagerRunnerImpl. This closes #9788. --- .../DefaultJobManagerRunnerFactory.java | 5 +- .../flink/runtime/dispatcher/Dispatcher.java | 12 +- .../runtime/jobmaster/JobManagerRunner.java | 418 +--------------- .../jobmaster/JobManagerRunnerImpl.java | 445 ++++++++++++++++++ .../TestingJobManagerRunnerFactory.java | 5 +- ...est.java => JobManagerRunnerImplTest.java} | 18 +- .../jobmaster/TestingJobManagerRunner.java | 76 +++ 7 files changed, 562 insertions(+), 417 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java rename flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/{JobManagerRunnerTest.java => JobManagerRunnerImplTest.java} (93%) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DefaultJobManagerRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DefaultJobManagerRunnerFactory.java index cab264d671..e42643c13a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DefaultJobManagerRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DefaultJobManagerRunnerFactory.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobManagerRunnerImpl; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobMasterConfiguration; import org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory; @@ -39,7 +40,7 @@ import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.apache.flink.runtime.shuffle.ShuffleServiceLoader; /** - * Singleton default factory for {@link JobManagerRunner}. + * Singleton default factory for {@link JobManagerRunnerImpl}. */ public enum DefaultJobManagerRunnerFactory implements JobManagerRunnerFactory { INSTANCE; @@ -75,7 +76,7 @@ public enum DefaultJobManagerRunnerFactory implements JobManagerRunnerFactory { schedulerNGFactory, shuffleMaster); - return new JobManagerRunner( + return new JobManagerRunnerImpl( jobGraph, jobMasterFactory, highAvailabilityServices, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 96f1e912d1..e3bfefb146 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -41,6 +41,7 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobManagerRunnerImpl; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.runtime.jobmaster.JobNotFinishedException; @@ -378,14 +379,15 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme } private JobManagerRunner startJobManagerRunner(JobManagerRunner jobManagerRunner) throws Exception { - final JobID jobId = jobManagerRunner.getJobGraph().getJobID(); + final JobID jobId = jobManagerRunner.getJobID(); FutureUtils.assertNoException( jobManagerRunner.getResultFuture().handleAsync( (ArchivedExecutionGraph archivedExecutionGraph, Throwable throwable) -> { // check if we are still the active JobManagerRunner by checking the identity - final CompletableFuture jobManagerRunnerFuture = jobManagerRunnerFutures.get(jobId); - final JobManagerRunner currentJobManagerRunner = jobManagerRunnerFuture != null ? jobManagerRunnerFuture.getNow(null) : null; + final JobManagerRunner currentJobManagerRunner = Optional.ofNullable(jobManagerRunnerFutures.get(jobId)) + .map(future -> future.getNow(null)) + .orElse(null); //noinspection ObjectEquality if (jobManagerRunner == currentJobManagerRunner) { if (archivedExecutionGraph != null) { @@ -682,7 +684,7 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme } /** - * Terminate all currently running {@link JobManagerRunner}. + * Terminate all currently running {@link JobManagerRunnerImpl}. */ private void terminateJobManagerRunners() { log.info("Stopping all currently running jobs of dispatcher {}.", getAddress()); @@ -809,7 +811,7 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme if (jobManagerRunnerFuture == null) { return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); } else { - final CompletableFuture leaderGatewayFuture = jobManagerRunnerFuture.thenCompose(JobManagerRunner::getLeaderGatewayFuture); + final CompletableFuture leaderGatewayFuture = jobManagerRunnerFuture.thenCompose(JobManagerRunner::getJobMasterGateway); return leaderGatewayFuture.thenApplyAsync( (JobMasterGateway jobMasterGateway) -> { // check whether the retrieved JobMasterGateway belongs still to a running JobMaster diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java index 846018b96a..4a1d1c7a13 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java @@ -18,424 +18,44 @@ package org.apache.flink.runtime.jobmaster; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; +import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.OnCompletionActions; -import org.apache.flink.runtime.jobmaster.factories.JobMasterServiceFactory; -import org.apache.flink.runtime.leaderelection.LeaderContender; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.util.AutoCloseableAsync; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.FlinkException; -import org.apache.flink.util.function.FunctionUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nonnull; - -import java.io.IOException; -import java.util.UUID; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionStage; -import java.util.concurrent.Executor; - -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; /** - * The runner for the job manager. It deals with job level leader election and make underlying job manager - * properly reacted. + * Interface for a runner which executes a {@link JobMaster}. */ -public class JobManagerRunner implements LeaderContender, OnCompletionActions, AutoCloseableAsync { - - private static final Logger log = LoggerFactory.getLogger(JobManagerRunner.class); - - // ------------------------------------------------------------------------ - - /** Lock to ensure that this runner can deal with leader election event and job completion notifies simultaneously. */ - private final Object lock = new Object(); - - /** The job graph needs to run. */ - private final JobGraph jobGraph; - - /** Used to check whether a job needs to be run. */ - private final RunningJobsRegistry runningJobsRegistry; - - /** Leader election for this job. */ - private final LeaderElectionService leaderElectionService; - - private final LibraryCacheManager libraryCacheManager; - - private final Executor executor; - - private final JobMasterService jobMasterService; - - private final FatalErrorHandler fatalErrorHandler; - - private final CompletableFuture resultFuture; - - private final CompletableFuture terminationFuture; - - private CompletableFuture leadershipOperation; - - /** flag marking the runner as shut down. */ - private volatile boolean shutdown; - - private volatile CompletableFuture leaderGatewayFuture; - - // ------------------------------------------------------------------------ +public interface JobManagerRunner extends AutoCloseableAsync { /** - * Exceptions that occur while creating the JobManager or JobManagerRunner are directly - * thrown and not reported to the given {@code FatalErrorHandler}. + * Start the execution of the {@link JobMaster}. * - * @throws Exception Thrown if the runner cannot be set up, because either one of the - * required services could not be started, or the Job could not be initialized. + * @throws Exception if the JobMaster cannot be started */ - public JobManagerRunner( - final JobGraph jobGraph, - final JobMasterServiceFactory jobMasterFactory, - final HighAvailabilityServices haServices, - final LibraryCacheManager libraryCacheManager, - final Executor executor, - final FatalErrorHandler fatalErrorHandler) throws Exception { - - this.resultFuture = new CompletableFuture<>(); - this.terminationFuture = new CompletableFuture<>(); - this.leadershipOperation = CompletableFuture.completedFuture(null); - - // make sure we cleanly shut down out JobManager services if initialization fails - try { - this.jobGraph = checkNotNull(jobGraph); - this.libraryCacheManager = checkNotNull(libraryCacheManager); - this.executor = checkNotNull(executor); - this.fatalErrorHandler = checkNotNull(fatalErrorHandler); - - checkArgument(jobGraph.getNumberOfVertices() > 0, "The given job is empty"); - - // libraries and class loader first - try { - libraryCacheManager.registerJob( - jobGraph.getJobID(), jobGraph.getUserJarBlobKeys(), jobGraph.getClasspaths()); - } catch (IOException e) { - throw new Exception("Cannot set up the user code libraries: " + e.getMessage(), e); - } - - final ClassLoader userCodeLoader = libraryCacheManager.getClassLoader(jobGraph.getJobID()); - if (userCodeLoader == null) { - throw new Exception("The user code class loader could not be initialized."); - } - - // high availability services next - this.runningJobsRegistry = haServices.getRunningJobsRegistry(); - this.leaderElectionService = haServices.getJobManagerLeaderElectionService(jobGraph.getJobID()); - - this.leaderGatewayFuture = new CompletableFuture<>(); - - // now start the JobManager - this.jobMasterService = jobMasterFactory.createJobMasterService(jobGraph, this, userCodeLoader); - } - catch (Throwable t) { - terminationFuture.completeExceptionally(t); - resultFuture.completeExceptionally(t); - - throw new JobExecutionException(jobGraph.getJobID(), "Could not set up JobManager", t); - } - } - - //---------------------------------------------------------------------------------------------- - // Getter - //---------------------------------------------------------------------------------------------- - - public CompletableFuture getLeaderGatewayFuture() { - return leaderGatewayFuture; - } - - public JobGraph getJobGraph() { - return jobGraph; - } - - public CompletableFuture getResultFuture() { - return resultFuture; - } - - //---------------------------------------------------------------------------------------------- - // Lifecycle management - //---------------------------------------------------------------------------------------------- - - public void start() throws Exception { - try { - leaderElectionService.start(this); - } catch (Exception e) { - log.error("Could not start the JobManager because the leader election service did not start.", e); - throw new Exception("Could not start the leader election service.", e); - } - } - - @Override - public CompletableFuture closeAsync() { - synchronized (lock) { - if (!shutdown) { - shutdown = true; - - setNewLeaderGatewayFuture(); - leaderGatewayFuture.completeExceptionally(new FlinkException("JobMaster has been shut down.")); - - final CompletableFuture jobManagerTerminationFuture = jobMasterService.closeAsync(); - - jobManagerTerminationFuture.whenComplete( - (Void ignored, Throwable throwable) -> { - try { - leaderElectionService.stop(); - } catch (Throwable t) { - throwable = ExceptionUtils.firstOrSuppressed(t, ExceptionUtils.stripCompletionException(throwable)); - } - - libraryCacheManager.unregisterJob(jobGraph.getJobID()); - - if (throwable != null) { - terminationFuture.completeExceptionally( - new FlinkException("Could not properly shut down the JobManagerRunner", throwable)); - } else { - terminationFuture.complete(null); - } - }); - - terminationFuture.whenComplete( - (Void ignored, Throwable throwable) -> { - resultFuture.completeExceptionally(new JobNotFinishedException(jobGraph.getJobID())); - }); - } - - return terminationFuture; - } - } - - //---------------------------------------------------------------------------------------------- - // Result and error handling methods - //---------------------------------------------------------------------------------------------- + void start() throws Exception; /** - * Job completion notification triggered by JobManager. + * Get the {@link JobMasterGateway} of the {@link JobMaster}. The future is + * only completed if the JobMaster becomes leader. + * + * @return Future with the JobMasterGateway once the underlying JobMaster becomes leader */ - @Override - public void jobReachedGloballyTerminalState(ArchivedExecutionGraph executionGraph) { - unregisterJobFromHighAvailability(); - // complete the result future with the terminal execution graph - resultFuture.complete(executionGraph); - } + CompletableFuture getJobMasterGateway(); /** - * Job completion notification triggered by self. + * Get the result future of this runner. The future is completed once the executed + * job reaches a globally terminal state. + * + * @return Future which is completed with the job result */ - @Override - public void jobFinishedByOther() { - resultFuture.completeExceptionally(new JobNotFinishedException(jobGraph.getJobID())); - } - - @Override - public void jobMasterFailed(Throwable cause) { - handleJobManagerRunnerError(cause); - } - - private void handleJobManagerRunnerError(Throwable cause) { - if (ExceptionUtils.isJvmFatalError(cause)) { - fatalErrorHandler.onFatalError(cause); - } else { - resultFuture.completeExceptionally(cause); - } - } + CompletableFuture getResultFuture(); /** - * Marks this runner's job as not running. Other JobManager will not recover the job - * after this call. + * Get the job id of the executed job. * - *

This method never throws an exception. + * @return job id of the executed job */ - private void unregisterJobFromHighAvailability() { - try { - runningJobsRegistry.setJobFinished(jobGraph.getJobID()); - } - catch (Throwable t) { - log.error("Could not un-register from high-availability services job {} ({})." + - "Other JobManager's may attempt to recover it and re-execute it.", - jobGraph.getName(), jobGraph.getJobID(), t); - } - } - - //---------------------------------------------------------------------------------------------- - // Leadership methods - //---------------------------------------------------------------------------------------------- - - @Override - public void grantLeadership(final UUID leaderSessionID) { - synchronized (lock) { - if (shutdown) { - log.info("JobManagerRunner already shutdown."); - return; - } - - leadershipOperation = leadershipOperation.thenCompose( - (ignored) -> { - synchronized (lock) { - return verifyJobSchedulingStatusAndStartJobManager(leaderSessionID); - } - }); - - handleException(leadershipOperation, "Could not start the job manager."); - } - } - - private CompletableFuture verifyJobSchedulingStatusAndStartJobManager(UUID leaderSessionId) { - final CompletableFuture jobSchedulingStatusFuture = getJobSchedulingStatus(); - - return jobSchedulingStatusFuture.thenCompose( - jobSchedulingStatus -> { - if (jobSchedulingStatus == JobSchedulingStatus.DONE) { - return jobAlreadyDone(); - } else { - return startJobMaster(leaderSessionId); - } - }); - } - - private CompletionStage startJobMaster(UUID leaderSessionId) { - log.info("JobManager runner for job {} ({}) was granted leadership with session id {} at {}.", - jobGraph.getName(), jobGraph.getJobID(), leaderSessionId, getAddress()); - - try { - runningJobsRegistry.setJobRunning(jobGraph.getJobID()); - } catch (IOException e) { - return FutureUtils.completedExceptionally( - new FlinkException( - String.format("Failed to set the job %s to running in the running jobs registry.", jobGraph.getJobID()), - e)); - } - - final CompletableFuture startFuture; - try { - startFuture = jobMasterService.start(new JobMasterId(leaderSessionId)); - } catch (Exception e) { - return FutureUtils.completedExceptionally(new FlinkException("Failed to start the JobMaster.", e)); - } - - final CompletableFuture currentLeaderGatewayFuture = leaderGatewayFuture; - return startFuture.thenAcceptAsync( - (Acknowledge ack) -> confirmLeaderSessionIdIfStillLeader(leaderSessionId, currentLeaderGatewayFuture), - executor); - } - - @Nonnull - private CompletionStage jobAlreadyDone() { - log.info("Granted leader ship but job {} has been finished. ", jobGraph.getJobID()); - jobFinishedByOther(); - return CompletableFuture.completedFuture(null); - } - - private CompletableFuture getJobSchedulingStatus() { - try { - JobSchedulingStatus jobSchedulingStatus = runningJobsRegistry.getJobSchedulingStatus(jobGraph.getJobID()); - return CompletableFuture.completedFuture(jobSchedulingStatus); - } catch (IOException e) { - return FutureUtils.completedExceptionally( - new FlinkException( - String.format("Could not retrieve the job scheduling status for job %s.", jobGraph.getJobID()), - e)); - } - } - - private void confirmLeaderSessionIdIfStillLeader(UUID leaderSessionId, CompletableFuture currentLeaderGatewayFuture) { - if (leaderElectionService.hasLeadership(leaderSessionId)) { - currentLeaderGatewayFuture.complete(jobMasterService.getGateway()); - leaderElectionService.confirmLeaderSessionID(leaderSessionId); - } else { - log.debug("Ignoring confirmation of leader session id because {} is no longer the leader.", getAddress()); - } - } - - @Override - public void revokeLeadership() { - synchronized (lock) { - if (shutdown) { - log.info("JobManagerRunner already shutdown."); - return; - } - - leadershipOperation = leadershipOperation.thenCompose( - (ignored) -> { - synchronized (lock) { - return revokeJobMasterLeadership(); - } - }); - - handleException(leadershipOperation, "Could not suspend the job manager."); - } - } - - private CompletableFuture revokeJobMasterLeadership() { - log.info("JobManager for job {} ({}) was revoked leadership at {}.", - jobGraph.getName(), jobGraph.getJobID(), getAddress()); - - setNewLeaderGatewayFuture(); - - return jobMasterService - .suspend(new FlinkException("JobManager is no longer the leader.")) - .thenApply(FunctionUtils.nullFn()); - } - - private void handleException(CompletableFuture leadershipOperation, String message) { - leadershipOperation.whenComplete( - (ignored, throwable) -> { - if (throwable != null) { - handleJobManagerRunnerError(new FlinkException(message, throwable)); - } - }); - } - - private void setNewLeaderGatewayFuture() { - final CompletableFuture oldLeaderGatewayFuture = leaderGatewayFuture; - - leaderGatewayFuture = new CompletableFuture<>(); - - if (!oldLeaderGatewayFuture.isDone()) { - leaderGatewayFuture.whenComplete( - (JobMasterGateway jobMasterGateway, Throwable throwable) -> { - if (throwable != null) { - oldLeaderGatewayFuture.completeExceptionally(throwable); - } else { - oldLeaderGatewayFuture.complete(jobMasterGateway); - } - }); - } - } - - @Override - public String getAddress() { - return jobMasterService.getAddress(); - } - - @Override - public void handleError(Exception exception) { - log.error("Leader Election Service encountered a fatal error.", exception); - handleJobManagerRunnerError(exception); - } - - //---------------------------------------------------------------------------------------------- - // Testing - //---------------------------------------------------------------------------------------------- - - @VisibleForTesting - boolean isShutdown() { - return shutdown; - } + JobID getJobID(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java new file mode 100644 index 0000000000..7b2609a65d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java @@ -0,0 +1,445 @@ +/* + * 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.jobmaster; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.OnCompletionActions; +import org.apache.flink.runtime.jobmaster.factories.JobMasterServiceFactory; +import org.apache.flink.runtime.leaderelection.LeaderContender; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.function.FunctionUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.io.IOException; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.Executor; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The runner for the job manager. It deals with job level leader election and make underlying job manager + * properly reacted. + */ +public class JobManagerRunnerImpl implements LeaderContender, OnCompletionActions, JobManagerRunner { + + private static final Logger log = LoggerFactory.getLogger(JobManagerRunnerImpl.class); + + // ------------------------------------------------------------------------ + + /** Lock to ensure that this runner can deal with leader election event and job completion notifies simultaneously. */ + private final Object lock = new Object(); + + /** The job graph needs to run. */ + private final JobGraph jobGraph; + + /** Used to check whether a job needs to be run. */ + private final RunningJobsRegistry runningJobsRegistry; + + /** Leader election for this job. */ + private final LeaderElectionService leaderElectionService; + + private final LibraryCacheManager libraryCacheManager; + + private final Executor executor; + + private final JobMasterService jobMasterService; + + private final FatalErrorHandler fatalErrorHandler; + + private final CompletableFuture resultFuture; + + private final CompletableFuture terminationFuture; + + private CompletableFuture leadershipOperation; + + /** flag marking the runner as shut down. */ + private volatile boolean shutdown; + + private volatile CompletableFuture leaderGatewayFuture; + + // ------------------------------------------------------------------------ + + /** + * Exceptions that occur while creating the JobManager or JobManagerRunnerImpl are directly + * thrown and not reported to the given {@code FatalErrorHandler}. + * + * @throws Exception Thrown if the runner cannot be set up, because either one of the + * required services could not be started, or the Job could not be initialized. + */ + public JobManagerRunnerImpl( + final JobGraph jobGraph, + final JobMasterServiceFactory jobMasterFactory, + final HighAvailabilityServices haServices, + final LibraryCacheManager libraryCacheManager, + final Executor executor, + final FatalErrorHandler fatalErrorHandler) throws Exception { + + this.resultFuture = new CompletableFuture<>(); + this.terminationFuture = new CompletableFuture<>(); + this.leadershipOperation = CompletableFuture.completedFuture(null); + + // make sure we cleanly shut down out JobManager services if initialization fails + try { + this.jobGraph = checkNotNull(jobGraph); + this.libraryCacheManager = checkNotNull(libraryCacheManager); + this.executor = checkNotNull(executor); + this.fatalErrorHandler = checkNotNull(fatalErrorHandler); + + checkArgument(jobGraph.getNumberOfVertices() > 0, "The given job is empty"); + + // libraries and class loader first + try { + libraryCacheManager.registerJob( + jobGraph.getJobID(), jobGraph.getUserJarBlobKeys(), jobGraph.getClasspaths()); + } catch (IOException e) { + throw new Exception("Cannot set up the user code libraries: " + e.getMessage(), e); + } + + final ClassLoader userCodeLoader = libraryCacheManager.getClassLoader(jobGraph.getJobID()); + if (userCodeLoader == null) { + throw new Exception("The user code class loader could not be initialized."); + } + + // high availability services next + this.runningJobsRegistry = haServices.getRunningJobsRegistry(); + this.leaderElectionService = haServices.getJobManagerLeaderElectionService(jobGraph.getJobID()); + + this.leaderGatewayFuture = new CompletableFuture<>(); + + // now start the JobManager + this.jobMasterService = jobMasterFactory.createJobMasterService(jobGraph, this, userCodeLoader); + } + catch (Throwable t) { + terminationFuture.completeExceptionally(t); + resultFuture.completeExceptionally(t); + + throw new JobExecutionException(jobGraph.getJobID(), "Could not set up JobManager", t); + } + } + + //---------------------------------------------------------------------------------------------- + // Getter + //---------------------------------------------------------------------------------------------- + + @Override + public CompletableFuture getJobMasterGateway() { + return leaderGatewayFuture; + } + + @Override + public CompletableFuture getResultFuture() { + return resultFuture; + } + + @Override + public JobID getJobID() { + return jobGraph.getJobID(); + } + + //---------------------------------------------------------------------------------------------- + // Lifecycle management + //---------------------------------------------------------------------------------------------- + + @Override + public void start() throws Exception { + try { + leaderElectionService.start(this); + } catch (Exception e) { + log.error("Could not start the JobManager because the leader election service did not start.", e); + throw new Exception("Could not start the leader election service.", e); + } + } + + @Override + public CompletableFuture closeAsync() { + synchronized (lock) { + if (!shutdown) { + shutdown = true; + + setNewLeaderGatewayFuture(); + leaderGatewayFuture.completeExceptionally(new FlinkException("JobMaster has been shut down.")); + + final CompletableFuture jobManagerTerminationFuture = jobMasterService.closeAsync(); + + jobManagerTerminationFuture.whenComplete( + (Void ignored, Throwable throwable) -> { + try { + leaderElectionService.stop(); + } catch (Throwable t) { + throwable = ExceptionUtils.firstOrSuppressed(t, ExceptionUtils.stripCompletionException(throwable)); + } + + libraryCacheManager.unregisterJob(jobGraph.getJobID()); + + if (throwable != null) { + terminationFuture.completeExceptionally( + new FlinkException("Could not properly shut down the JobManagerRunner", throwable)); + } else { + terminationFuture.complete(null); + } + }); + + terminationFuture.whenComplete( + (Void ignored, Throwable throwable) -> { + resultFuture.completeExceptionally(new JobNotFinishedException(jobGraph.getJobID())); + }); + } + + return terminationFuture; + } + } + + //---------------------------------------------------------------------------------------------- + // Result and error handling methods + //---------------------------------------------------------------------------------------------- + + /** + * Job completion notification triggered by JobManager. + */ + @Override + public void jobReachedGloballyTerminalState(ArchivedExecutionGraph executionGraph) { + unregisterJobFromHighAvailability(); + // complete the result future with the terminal execution graph + resultFuture.complete(executionGraph); + } + + /** + * Job completion notification triggered by self. + */ + @Override + public void jobFinishedByOther() { + resultFuture.completeExceptionally(new JobNotFinishedException(jobGraph.getJobID())); + } + + @Override + public void jobMasterFailed(Throwable cause) { + handleJobManagerRunnerError(cause); + } + + private void handleJobManagerRunnerError(Throwable cause) { + if (ExceptionUtils.isJvmFatalError(cause)) { + fatalErrorHandler.onFatalError(cause); + } else { + resultFuture.completeExceptionally(cause); + } + } + + /** + * Marks this runner's job as not running. Other JobManager will not recover the job + * after this call. + * + *

This method never throws an exception. + */ + private void unregisterJobFromHighAvailability() { + try { + runningJobsRegistry.setJobFinished(jobGraph.getJobID()); + } + catch (Throwable t) { + log.error("Could not un-register from high-availability services job {} ({})." + + "Other JobManager's may attempt to recover it and re-execute it.", + jobGraph.getName(), jobGraph.getJobID(), t); + } + } + + //---------------------------------------------------------------------------------------------- + // Leadership methods + //---------------------------------------------------------------------------------------------- + + @Override + public void grantLeadership(final UUID leaderSessionID) { + synchronized (lock) { + if (shutdown) { + log.info("JobManagerRunner already shutdown."); + return; + } + + leadershipOperation = leadershipOperation.thenCompose( + (ignored) -> { + synchronized (lock) { + return verifyJobSchedulingStatusAndStartJobManager(leaderSessionID); + } + }); + + handleException(leadershipOperation, "Could not start the job manager."); + } + } + + private CompletableFuture verifyJobSchedulingStatusAndStartJobManager(UUID leaderSessionId) { + final CompletableFuture jobSchedulingStatusFuture = getJobSchedulingStatus(); + + return jobSchedulingStatusFuture.thenCompose( + jobSchedulingStatus -> { + if (jobSchedulingStatus == JobSchedulingStatus.DONE) { + return jobAlreadyDone(); + } else { + return startJobMaster(leaderSessionId); + } + }); + } + + private CompletionStage startJobMaster(UUID leaderSessionId) { + log.info("JobManager runner for job {} ({}) was granted leadership with session id {} at {}.", + jobGraph.getName(), jobGraph.getJobID(), leaderSessionId, getAddress()); + + try { + runningJobsRegistry.setJobRunning(jobGraph.getJobID()); + } catch (IOException e) { + return FutureUtils.completedExceptionally( + new FlinkException( + String.format("Failed to set the job %s to running in the running jobs registry.", jobGraph.getJobID()), + e)); + } + + final CompletableFuture startFuture; + try { + startFuture = jobMasterService.start(new JobMasterId(leaderSessionId)); + } catch (Exception e) { + return FutureUtils.completedExceptionally(new FlinkException("Failed to start the JobMaster.", e)); + } + + final CompletableFuture currentLeaderGatewayFuture = leaderGatewayFuture; + return startFuture.thenAcceptAsync( + (Acknowledge ack) -> confirmLeaderSessionIdIfStillLeader(leaderSessionId, currentLeaderGatewayFuture), + executor); + } + + @Nonnull + private CompletionStage jobAlreadyDone() { + log.info("Granted leader ship but job {} has been finished. ", jobGraph.getJobID()); + jobFinishedByOther(); + return CompletableFuture.completedFuture(null); + } + + private CompletableFuture getJobSchedulingStatus() { + try { + JobSchedulingStatus jobSchedulingStatus = runningJobsRegistry.getJobSchedulingStatus(jobGraph.getJobID()); + return CompletableFuture.completedFuture(jobSchedulingStatus); + } catch (IOException e) { + return FutureUtils.completedExceptionally( + new FlinkException( + String.format("Could not retrieve the job scheduling status for job %s.", jobGraph.getJobID()), + e)); + } + } + + private void confirmLeaderSessionIdIfStillLeader(UUID leaderSessionId, CompletableFuture currentLeaderGatewayFuture) { + if (leaderElectionService.hasLeadership(leaderSessionId)) { + currentLeaderGatewayFuture.complete(jobMasterService.getGateway()); + leaderElectionService.confirmLeaderSessionID(leaderSessionId); + } else { + log.debug("Ignoring confirmation of leader session id because {} is no longer the leader.", getAddress()); + } + } + + @Override + public void revokeLeadership() { + synchronized (lock) { + if (shutdown) { + log.info("JobManagerRunner already shutdown."); + return; + } + + leadershipOperation = leadershipOperation.thenCompose( + (ignored) -> { + synchronized (lock) { + return revokeJobMasterLeadership(); + } + }); + + handleException(leadershipOperation, "Could not suspend the job manager."); + } + } + + private CompletableFuture revokeJobMasterLeadership() { + log.info("JobManager for job {} ({}) was revoked leadership at {}.", + jobGraph.getName(), jobGraph.getJobID(), getAddress()); + + setNewLeaderGatewayFuture(); + + return jobMasterService + .suspend(new FlinkException("JobManager is no longer the leader.")) + .thenApply(FunctionUtils.nullFn()); + } + + private void handleException(CompletableFuture leadershipOperation, String message) { + leadershipOperation.whenComplete( + (ignored, throwable) -> { + if (throwable != null) { + handleJobManagerRunnerError(new FlinkException(message, throwable)); + } + }); + } + + private void setNewLeaderGatewayFuture() { + final CompletableFuture oldLeaderGatewayFuture = leaderGatewayFuture; + + leaderGatewayFuture = new CompletableFuture<>(); + + if (!oldLeaderGatewayFuture.isDone()) { + leaderGatewayFuture.whenComplete( + (JobMasterGateway jobMasterGateway, Throwable throwable) -> { + if (throwable != null) { + oldLeaderGatewayFuture.completeExceptionally(throwable); + } else { + oldLeaderGatewayFuture.complete(jobMasterGateway); + } + }); + } + } + + @Override + public String getAddress() { + return jobMasterService.getAddress(); + } + + @Override + public void handleError(Exception exception) { + log.error("Leader Election Service encountered a fatal error.", exception); + handleJobManagerRunnerError(exception); + } + + //---------------------------------------------------------------------------------------------- + // Testing + //---------------------------------------------------------------------------------------------- + + @VisibleForTesting + boolean isShutdown() { + return shutdown; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java index c19038c5e9..ea2580d021 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobManagerRunnerImpl; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -82,10 +83,10 @@ class TestingJobManagerRunnerFactory implements JobManagerRunnerFactory { } else { jobGraphFuture.complete(jobGraph); - final JobManagerRunner mock = mock(JobManagerRunner.class); + final JobManagerRunnerImpl mock = mock(JobManagerRunnerImpl.class); when(mock.getResultFuture()).thenReturn(resultFuture); when(mock.closeAsync()).thenReturn(terminationFuture); - when(mock.getJobGraph()).thenReturn(jobGraph); + when(mock.getJobID()).thenReturn(jobGraph.getJobID()); return mock; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImplTest.java similarity index 93% rename from flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImplTest.java index b560600e6a..23c4e31f17 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImplTest.java @@ -64,9 +64,9 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; /** - * Tests for the {@link JobManagerRunner}. + * Tests for the {@link JobManagerRunnerImpl}. */ -public class JobManagerRunnerTest extends TestLogger { +public class JobManagerRunnerImplTest extends TestLogger { @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -129,7 +129,7 @@ public class JobManagerRunnerTest extends TestLogger { @Test public void testJobCompletion() throws Exception { - final JobManagerRunner jobManagerRunner = createJobManagerRunner(); + final JobManagerRunnerImpl jobManagerRunner = createJobManagerRunner(); try { jobManagerRunner.start(); @@ -148,7 +148,7 @@ public class JobManagerRunnerTest extends TestLogger { @Test public void testJobFinishedByOther() throws Exception { - final JobManagerRunner jobManagerRunner = createJobManagerRunner(); + final JobManagerRunnerImpl jobManagerRunner = createJobManagerRunner(); try { jobManagerRunner.start(); @@ -217,7 +217,7 @@ public class JobManagerRunnerTest extends TestLogger { } /** - * Tests that the {@link JobManagerRunner} always waits for the previous leadership operation + * Tests that the {@link JobManagerRunnerImpl} always waits for the previous leadership operation * (granting or revoking leadership) to finish before starting a new leadership operation. */ @Test @@ -254,7 +254,7 @@ public class JobManagerRunnerTest extends TestLogger { } /** - * Tests that the {@link JobManagerRunner} always waits for the previous leadership operation + * Tests that the {@link JobManagerRunnerImpl} always waits for the previous leadership operation * (granting or revoking leadership) to finish before starting a new leadership operation. */ @Test @@ -299,7 +299,7 @@ public class JobManagerRunnerTest extends TestLogger { } @Nonnull - private JobManagerRunner createJobManagerRunner() throws Exception { + private JobManagerRunnerImpl createJobManagerRunner() throws Exception { return createJobManagerRunner(defaultJobMasterServiceFactory, libraryCacheManager); } @@ -309,8 +309,8 @@ public class JobManagerRunnerTest extends TestLogger { } @Nonnull - private JobManagerRunner createJobManagerRunner(JobMasterServiceFactory jobMasterServiceFactory, LibraryCacheManager libraryCacheManager) throws Exception{ - return new JobManagerRunner( + private JobManagerRunnerImpl createJobManagerRunner(JobMasterServiceFactory jobMasterServiceFactory, LibraryCacheManager libraryCacheManager) throws Exception{ + return new JobManagerRunnerImpl( jobGraph, jobMasterServiceFactory, haServices, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java new file mode 100644 index 0000000000..fc29f0784a --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.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.runtime.jobmaster; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; + +import java.util.concurrent.CompletableFuture; + +/** + * Testing implementation of the {@link JobManagerRunner}. + */ +public class TestingJobManagerRunner implements JobManagerRunner { + + private final JobID jobId; + + private final CompletableFuture resultFuture; + + private final CompletableFuture jobMasterGatewayFuture; + + private final CompletableFuture terminationFuture; + + public TestingJobManagerRunner(JobID jobId) { + this.jobId = jobId; + this.resultFuture = new CompletableFuture<>(); + this.jobMasterGatewayFuture = new CompletableFuture<>(); + this.terminationFuture = new CompletableFuture<>(); + + terminationFuture.whenComplete((ignored, ignoredThrowable) -> resultFuture.completeExceptionally(new JobNotFinishedException(jobId))); + } + + @Override + public void start() throws Exception {} + + @Override + public CompletableFuture getJobMasterGateway() { + return jobMasterGatewayFuture; + } + + @Override + public CompletableFuture getResultFuture() { + return resultFuture; + } + + @Override + public JobID getJobID() { + return jobId; + } + + @Override + public CompletableFuture closeAsync() { + terminationFuture.complete(null); + + return terminationFuture; + } + + public void completeResultFuture(ArchivedExecutionGraph archivedExecutionGraph) { + resultFuture.complete(archivedExecutionGraph); + } +} -- Gitee From 5cace47252cdcaa4da40760ffe61b0fb1be1aad9 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 30 Sep 2019 14:46:03 +0200 Subject: [PATCH 096/268] [hotfix][benchmarks] Expose RecordWriterBuilder from the benchmark environment --- .../StreamNetworkBenchmarkEnvironment.java | 49 +++++++------------ .../StreamNetworkPointToPointBenchmark.java | 6 ++- .../StreamNetworkThroughputBenchmark.java | 6 ++- 3 files changed, 29 insertions(+), 32 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java index 8351b3ce7a..6cdb229b5c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java @@ -27,8 +27,6 @@ import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.TaskEventDispatcher; -import org.apache.flink.runtime.io.network.api.writer.RecordWriter; -import org.apache.flink.runtime.io.network.api.writer.RecordWriterBuilder; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.netty.NettyConfig; import org.apache.flink.runtime.io.network.partition.InputChannelTestUtils; @@ -178,9 +176,25 @@ public class StreamNetworkBenchmarkEnvironment { return receiver; } - public RecordWriter createRecordWriter(int partitionIndex, long flushTimeout) throws Exception { - ResultPartitionWriter sender = createResultPartition(jobId, partitionIds[partitionIndex], senderEnv, channels); - return new RecordWriterBuilder().setTimeout(flushTimeout).build(sender); + public ResultPartitionWriter createResultPartitionWriter(int partitionIndex) throws Exception { + + ResultPartitionWriter resultPartitionWriter = new ResultPartitionBuilder() + .setResultPartitionId(partitionIds[partitionIndex]) + .setResultPartitionType(ResultPartitionType.PIPELINED_BOUNDED) + .setNumberOfSubpartitions(channels) + .setResultPartitionManager(senderEnv.getResultPartitionManager()) + .setupBufferPoolFactoryFromNettyShuffleEnvironment(senderEnv) + .build(); + + ResultPartitionWriter consumableNotifyingPartitionWriter = new ConsumableNotifyingResultPartitionWriterDecorator( + new NoOpTaskActions(), + jobId, + resultPartitionWriter, + new NoOpResultPartitionConsumableNotifier()); + + consumableNotifyingPartitionWriter.setup(); + + return consumableNotifyingPartitionWriter; } private void generatePartitionIds() throws Exception { @@ -205,31 +219,6 @@ public class StreamNetworkBenchmarkEnvironment { .build(); } - protected ResultPartitionWriter createResultPartition( - JobID jobId, - ResultPartitionID partitionId, - NettyShuffleEnvironment environment, - int channels) throws Exception { - - ResultPartitionWriter resultPartitionWriter = new ResultPartitionBuilder() - .setResultPartitionId(partitionId) - .setResultPartitionType(ResultPartitionType.PIPELINED_BOUNDED) - .setNumberOfSubpartitions(channels) - .setResultPartitionManager(environment.getResultPartitionManager()) - .setupBufferPoolFactoryFromNettyShuffleEnvironment(environment) - .build(); - - ResultPartitionWriter consumableNotifyingPartitionWriter = new ConsumableNotifyingResultPartitionWriterDecorator( - new NoOpTaskActions(), - jobId, - resultPartitionWriter, - new NoOpResultPartitionConsumableNotifier()); - - consumableNotifyingPartitionWriter.setup(); - - return consumableNotifyingPartitionWriter; - } - private InputGate createInputGate(TaskManagerLocation senderLocation) throws Exception { InputGate[] gates = new InputGate[channels]; for (int channel = 0; channel < channels; ++channel) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java index bb6b9e2f16..df68d4da52 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java @@ -20,6 +20,8 @@ package org.apache.flink.streaming.runtime.io.benchmark; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; +import org.apache.flink.runtime.io.network.api.writer.RecordWriterBuilder; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.types.LongValue; import java.util.concurrent.CompletableFuture; @@ -77,7 +79,9 @@ public class StreamNetworkPointToPointBenchmark { environment = new StreamNetworkBenchmarkEnvironment<>(); environment.setUp(1, 1, false, false, -1, -1, config); - recordWriter = environment.createRecordWriter(0, flushTimeout); + ResultPartitionWriter resultPartitionWriter = environment.createResultPartitionWriter(0); + + recordWriter = new RecordWriterBuilder().setTimeout(flushTimeout).build(resultPartitionWriter); receiver = environment.createReceiver(); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java index b5d1c0729c..8b57fa4497 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java @@ -19,6 +19,8 @@ package org.apache.flink.streaming.runtime.io.benchmark; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.io.network.api.writer.RecordWriterBuilder; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.types.LongValue; import java.util.concurrent.CompletableFuture; @@ -112,8 +114,10 @@ public class StreamNetworkThroughputBenchmark { config); writerThreads = new LongRecordWriterThread[recordWriters]; for (int writer = 0; writer < recordWriters; writer++) { + ResultPartitionWriter resultPartitionWriter = environment.createResultPartitionWriter(writer); + RecordWriterBuilder recordWriterBuilder = new RecordWriterBuilder().setTimeout(flushTimeout); writerThreads[writer] = new LongRecordWriterThread( - environment.createRecordWriter(writer, flushTimeout), + recordWriterBuilder.build(resultPartitionWriter), broadcastMode); writerThreads[writer].start(); } -- Gitee From c552082df1421a73944bed6264bedd34dc846be9 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 30 Sep 2019 14:48:11 +0200 Subject: [PATCH 097/268] [hotfix][benchmarks] Remove unused field from benchmark environment --- .../io/benchmark/StreamNetworkBenchmarkEnvironment.java | 4 ---- .../io/benchmark/StreamNetworkPointToPointBenchmark.java | 2 +- .../io/benchmark/StreamNetworkThroughputBenchmark.java | 1 - 3 files changed, 1 insertion(+), 6 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java index 6cdb229b5c..aaeb1b0ef6 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java @@ -78,7 +78,6 @@ public class StreamNetworkBenchmarkEnvironment { protected NettyShuffleEnvironment receiverEnv; protected int channels; - protected boolean broadcastMode = false; protected boolean localMode = false; protected ResultPartitionID[] partitionIds; @@ -96,7 +95,6 @@ public class StreamNetworkBenchmarkEnvironment { setUp( writers, channels, - false, localMode, senderBufferPoolSize, receiverBufferPoolSize, @@ -120,12 +118,10 @@ public class StreamNetworkBenchmarkEnvironment { public void setUp( int writers, int channels, - boolean broadcastMode, boolean localMode, int senderBufferPoolSize, int receiverBufferPoolSize, Configuration config) throws Exception { - this.broadcastMode = broadcastMode; this.localMode = localMode; this.channels = channels; this.partitionIds = new ResultPartitionID[writers]; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java index df68d4da52..3277d50958 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java @@ -77,7 +77,7 @@ public class StreamNetworkPointToPointBenchmark { */ public void setUp(long flushTimeout, Configuration config) throws Exception { environment = new StreamNetworkBenchmarkEnvironment<>(); - environment.setUp(1, 1, false, false, -1, -1, config); + environment.setUp(1, 1, false, -1, -1, config); ResultPartitionWriter resultPartitionWriter = environment.createResultPartitionWriter(0); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java index 8b57fa4497..3596591c9a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java @@ -107,7 +107,6 @@ public class StreamNetworkThroughputBenchmark { environment.setUp( recordWriters, channels, - broadcastMode, localMode, senderBufferPoolSize, receiverBufferPoolSize, -- Gitee From 9f21c77b68afcd8033e3b5d370ef92b73ad4f811 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 30 Sep 2019 14:51:18 +0200 Subject: [PATCH 098/268] [FLINK-10995][benchmarks] Use BroadcastRecordWriter for network broadcast benchmarks --- .../io/benchmark/StreamNetworkThroughputBenchmark.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java index 3596591c9a..ed3a05f87c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java @@ -21,6 +21,7 @@ package org.apache.flink.streaming.runtime.io.benchmark; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.io.network.api.writer.RecordWriterBuilder; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; import org.apache.flink.types.LongValue; import java.util.concurrent.CompletableFuture; @@ -115,6 +116,9 @@ public class StreamNetworkThroughputBenchmark { for (int writer = 0; writer < recordWriters; writer++) { ResultPartitionWriter resultPartitionWriter = environment.createResultPartitionWriter(writer); RecordWriterBuilder recordWriterBuilder = new RecordWriterBuilder().setTimeout(flushTimeout); + if (broadcastMode) { + recordWriterBuilder.setChannelSelector(new BroadcastPartitioner()); + } writerThreads[writer] = new LongRecordWriterThread( recordWriterBuilder.build(resultPartitionWriter), broadcastMode); -- Gitee From abcdec1cae2b9be79ed28bd58b64849e6d52ecf7 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 7 May 2019 12:06:41 +0200 Subject: [PATCH 099/268] [FLINK-14260] Port MiniDispatcherTest to use new TestingJobManagerRunnerFactoryNG --- .../dispatcher/MiniDispatcherTest.java | 24 +++----- .../TestingJobManagerRunnerFactoryNG.java | 61 +++++++++++++++++++ 2 files changed, 70 insertions(+), 15 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java index f9266cc323..7630778a4a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; @@ -85,17 +86,13 @@ public class MiniDispatcherTest extends TestLogger { private final ArchivedExecutionGraphStore archivedExecutionGraphStore = new MemoryArchivedExecutionGraphStore(); - private CompletableFuture jobGraphFuture; - - private CompletableFuture resultFuture; - private TestingLeaderElectionService dispatcherLeaderElectionService; private TestingHighAvailabilityServices highAvailabilityServices; private TestingFatalErrorHandler testingFatalErrorHandler; - private TestingJobManagerRunnerFactory testingJobManagerRunnerFactory; + private TestingJobManagerRunnerFactoryNG testingJobManagerRunnerFactory; @BeforeClass public static void setupClass() throws IOException { @@ -122,10 +119,7 @@ public class MiniDispatcherTest extends TestLogger { highAvailabilityServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService); - jobGraphFuture = new CompletableFuture<>(); - resultFuture = new CompletableFuture<>(); - - testingJobManagerRunnerFactory = new TestingJobManagerRunnerFactory(jobGraphFuture, resultFuture, CompletableFuture.completedFuture(null)); + testingJobManagerRunnerFactory = new TestingJobManagerRunnerFactoryNG(); } @After @@ -158,9 +152,9 @@ public class MiniDispatcherTest extends TestLogger { // wait until the Dispatcher is the leader dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - final JobGraph actualJobGraph = jobGraphFuture.get(); + final TestingJobManagerRunner testingJobManagerRunner = testingJobManagerRunnerFactory.takeCreatedJobManagerRunner(); - assertThat(actualJobGraph.getJobID(), is(jobGraph.getJobID())); + assertThat(testingJobManagerRunner.getJobID(), is(jobGraph.getJobID())); } finally { RpcUtils.terminateRpcEndpoint(miniDispatcher, timeout); } @@ -181,9 +175,9 @@ public class MiniDispatcherTest extends TestLogger { dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); // wait until we have submitted the job - jobGraphFuture.get(); + final TestingJobManagerRunner testingJobManagerRunner = testingJobManagerRunnerFactory.takeCreatedJobManagerRunner(); - resultFuture.complete(archivedExecutionGraph); + testingJobManagerRunner.completeResultFuture(archivedExecutionGraph); // wait until we terminate miniDispatcher.getJobTerminationFuture().get(); @@ -207,9 +201,9 @@ public class MiniDispatcherTest extends TestLogger { dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); // wait until we have submitted the job - jobGraphFuture.get(); + final TestingJobManagerRunner testingJobManagerRunner = testingJobManagerRunnerFactory.takeCreatedJobManagerRunner(); - resultFuture.complete(archivedExecutionGraph); + testingJobManagerRunner.completeResultFuture(archivedExecutionGraph); assertFalse(miniDispatcher.getTerminationFuture().isDone()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java new file mode 100644 index 0000000000..5bc1e3c790 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.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.runtime.dispatcher; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; +import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; +import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; + +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; + +/** + * Testing implementation of {@link JobManagerRunnerFactory} which returns a {@link TestingJobManagerRunner}. + */ +public class TestingJobManagerRunnerFactoryNG implements JobManagerRunnerFactory { + + private final BlockingQueue createdJobManagerRunner = new ArrayBlockingQueue<>(16); + + @Override + public JobManagerRunner createJobManagerRunner( + JobGraph jobGraph, + Configuration configuration, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + JobManagerSharedServices jobManagerServices, + JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, + FatalErrorHandler fatalErrorHandler) { + final TestingJobManagerRunner testingJobManagerRunner = new TestingJobManagerRunner(jobGraph.getJobID()); + createdJobManagerRunner.offer(testingJobManagerRunner); + + return testingJobManagerRunner; + } + + public TestingJobManagerRunner takeCreatedJobManagerRunner() throws InterruptedException { + return createdJobManagerRunner.take(); + } +} -- Gitee From 5e2d301ef80067093e6e72184c01c6f5162240a7 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 7 May 2019 12:08:00 +0200 Subject: [PATCH 100/268] [FLINK-14260] Port DispatcherHATest to use TetingJobManagerRunnerFactoryNG --- .../org/apache/flink/runtime/dispatcher/DispatcherHATest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.java index 16840c0b70..713a6a2f11 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.java @@ -291,8 +291,8 @@ public class DispatcherHATest extends TestLogger { } @Nonnull - private TestingJobManagerRunnerFactory createTestingJobManagerRunnerFactory() { - return new TestingJobManagerRunnerFactory(new CompletableFuture<>(), new CompletableFuture<>(), CompletableFuture.completedFuture(null)); + private TestingJobManagerRunnerFactoryNG createTestingJobManagerRunnerFactory() { + return new TestingJobManagerRunnerFactoryNG(); } private HATestingDispatcher createDispatcher(HighAvailabilityServices haServices) throws Exception { -- Gitee From a321b53abf5d09ea13da2d1f83bc0183604c4f34 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 7 May 2019 15:42:33 +0200 Subject: [PATCH 101/268] [FLINK-14260] Port ZooKeeperHaDispatcherTest to use TestingJobManagerRunnerFactoryNG --- .../dispatcher/ZooKeeperHADispatcherTest.java | 30 ++++++++++--------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java index 0343caadf0..7c979692b0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java @@ -24,7 +24,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.blob.VoidBlobStore; -import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; @@ -33,6 +32,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobmanager.ZooKeeperJobGraphStore; import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; @@ -154,7 +154,7 @@ public class ZooKeeperHADispatcherTest extends TestLogger { final TestingDispatcher dispatcher = createDispatcher( testingHighAvailabilityServices, - new TestingJobManagerRunnerFactory(new CompletableFuture<>(), new CompletableFuture<>(), CompletableFuture.completedFuture(null))); + new TestingJobManagerRunnerFactoryNG()); dispatcher.start(); @@ -219,15 +219,14 @@ public class ZooKeeperHADispatcherTest extends TestLogger { final TestingLeaderElectionService leaderElectionService2 = new TestingLeaderElectionService(); haServices2.setDispatcherLeaderElectionService(leaderElectionService2); - final CompletableFuture jobGraphFuture = new CompletableFuture<>(); - final CompletableFuture resultFuture = new CompletableFuture<>(); + final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = new TestingJobManagerRunnerFactoryNG(); final TestingDispatcher dispatcher1 = createDispatcher( haServices1, - new TestingJobManagerRunnerFactory(jobGraphFuture, resultFuture, CompletableFuture.completedFuture(null))); + jobManagerRunnerFactory); final TestingDispatcher dispatcher2 = createDispatcher( haServices2, - new TestingJobManagerRunnerFactory(new CompletableFuture<>(), new CompletableFuture<>(), CompletableFuture.completedFuture(null))); + new TestingJobManagerRunnerFactoryNG()); try { dispatcher1.start(); @@ -242,10 +241,10 @@ public class ZooKeeperHADispatcherTest extends TestLogger { final CompletableFuture jobResultFuture = dispatcherGateway1.requestJobResult(jobGraph.getJobID(), TIMEOUT); - jobGraphFuture.get(); + final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); // complete the job - resultFuture.complete(new ArchivedExecutionGraphBuilder().setJobID(jobGraph.getJobID()).setState(JobStatus.FINISHED).build()); + testingJobManagerRunner.completeResultFuture(new ArchivedExecutionGraphBuilder().setJobID(jobGraph.getJobID()).setState(JobStatus.FINISHED).build()); final JobResult jobResult = jobResultFuture.get(); @@ -282,14 +281,13 @@ public class ZooKeeperHADispatcherTest extends TestLogger { try { haServices = new ZooKeeperHaServices(curatorFramework, rpcService.getExecutor(), configuration, new VoidBlobStore()); - final CompletableFuture jobGraphFuture1 = new CompletableFuture<>(); + final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = new TestingJobManagerRunnerFactoryNG(); dispatcher1 = createDispatcher( haServices, - new TestingJobManagerRunnerFactory(jobGraphFuture1, new CompletableFuture<>(), CompletableFuture.completedFuture(null))); - final CompletableFuture jobGraphFuture2 = new CompletableFuture<>(); + jobManagerRunnerFactory); dispatcher2 = createDispatcher( haServices, - new TestingJobManagerRunnerFactory(jobGraphFuture2, new CompletableFuture<>(), CompletableFuture.completedFuture(null))); + jobManagerRunnerFactory); dispatcher1.start(); dispatcher2.start(); @@ -303,13 +301,17 @@ public class ZooKeeperHADispatcherTest extends TestLogger { final JobGraph nonEmptyJobGraph = DispatcherHATest.createNonEmptyJobGraph(); dispatcherGateway.submitJob(nonEmptyJobGraph, TIMEOUT).get(); + // pop first job manager runner + jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + if (dispatcher1.getAddress().equals(leaderConnectionInfo.getAddress())) { dispatcher1.closeAsync(); - assertThat(jobGraphFuture2.get().getJobID(), is(equalTo(nonEmptyJobGraph.getJobID()))); } else { dispatcher2.closeAsync(); - assertThat(jobGraphFuture1.get().getJobID(), is(equalTo(nonEmptyJobGraph.getJobID()))); } + + final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + assertThat(testingJobManagerRunner.getJobID(), is(equalTo(nonEmptyJobGraph.getJobID()))); } finally { if (dispatcher1 != null) { RpcUtils.terminateRpcEndpoint(dispatcher1, TIMEOUT); -- Gitee From e3e3f161913168193832fa2483dc08a213b25428 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 7 May 2019 15:47:25 +0200 Subject: [PATCH 102/268] [FLINK-14260] Replace TestingJobManagerRunnerFactory with TestingJobManagerRunnerFactoryNG in DispatcherTest --- .../org/apache/flink/runtime/dispatcher/DispatcherTest.java | 4 +--- .../runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java | 2 +- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index dfeaac7377..daac79a311 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -764,14 +764,12 @@ public class DispatcherTest extends TestLogger { dispatcher.getTerminationFuture().get(); } - private final class BlockingJobManagerRunnerFactory extends TestingJobManagerRunnerFactory { + private final class BlockingJobManagerRunnerFactory extends TestingJobManagerRunnerFactoryNG { @Nonnull private final ThrowingRunnable jobManagerRunnerCreationLatch; BlockingJobManagerRunnerFactory(@Nonnull ThrowingRunnable jobManagerRunnerCreationLatch) { - super(new CompletableFuture<>(), new CompletableFuture<>(), CompletableFuture.completedFuture(null)); - this.jobManagerRunnerCreationLatch = jobManagerRunnerCreationLatch; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java index 5bc1e3c790..4e7c70fe7e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java @@ -48,7 +48,7 @@ public class TestingJobManagerRunnerFactoryNG implements JobManagerRunnerFactory HeartbeatServices heartbeatServices, JobManagerSharedServices jobManagerServices, JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, - FatalErrorHandler fatalErrorHandler) { + FatalErrorHandler fatalErrorHandler) throws Exception { final TestingJobManagerRunner testingJobManagerRunner = new TestingJobManagerRunner(jobGraph.getJobID()); createdJobManagerRunner.offer(testingJobManagerRunner); -- Gitee From 75ac9143eaf4769aed105ebea05601b26206b05b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 7 May 2019 16:27:38 +0200 Subject: [PATCH 103/268] [FLINK-14260] Port DispatcherResourceCleanupTest to use TestingJobManagerRunnerFactoryNG --- .../DispatcherResourceCleanupTest.java | 109 +++++++++++------- .../runtime/dispatcher/DispatcherTest.java | 3 +- .../TestingJobManagerRunnerFactoryNG.java | 31 ++++- .../jobmaster/TestingJobManagerRunner.java | 19 ++- 4 files changed, 113 insertions(+), 49 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 313db2a844..544137c803 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -33,17 +33,24 @@ import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.RunningJobsRegistry; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobNotFinishedException; +import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; +import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.util.TestingFatalErrorHandler; @@ -71,8 +78,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Supplier; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; @@ -120,13 +125,9 @@ public class DispatcherResourceCleanupTest extends TestLogger { private File blobFile; - private AtomicReference> failJobMasterCreationWith; - private CompletableFuture storedHABlobFuture; private CompletableFuture deleteAllHABlobsFuture; - private CompletableFuture resultFuture; private CompletableFuture cleanupJobFuture; - private CompletableFuture terminationFuture; private FaultyJobGraphStore submittedJobGraphStore; @BeforeClass @@ -164,7 +165,6 @@ public class DispatcherResourceCleanupTest extends TestLogger { .createTestingBlobStore(); cleanupJobFuture = new CompletableFuture<>(); - terminationFuture = new CompletableFuture<>(); blobServer = new TestingBlobServer(configuration, testingBlobStore, cleanupJobFuture); @@ -173,12 +173,27 @@ public class DispatcherResourceCleanupTest extends TestLogger { jobGraph.addUserJarBlobKey(permanentBlobKey); blobFile = blobServer.getStorageLocation(jobId, permanentBlobKey); - resultFuture = new CompletableFuture<>(); + assertThat(blobFile.exists(), is(true)); + + // verify that we stored the blob also in the BlobStore + assertThat(storedHABlobFuture.get(), equalTo(permanentBlobKey)); fatalErrorHandler = new TestingFatalErrorHandler(); + } + + private TestingJobManagerRunnerFactoryNG startDispatcherAndSubmitJob() throws Exception { + return startDispatcherAndSubmitJob(0); + } - failJobMasterCreationWith = new AtomicReference<>(); + private TestingJobManagerRunnerFactoryNG startDispatcherAndSubmitJob(int numBlockingJobManagerRunners) throws Exception { + final TestingJobManagerRunnerFactoryNG testingJobManagerRunnerFactoryNG = new TestingJobManagerRunnerFactoryNG(numBlockingJobManagerRunners); + startDispatcher(testingJobManagerRunnerFactoryNG); + submitJob(); + + return testingJobManagerRunnerFactoryNG; + } + private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); dispatcher = new TestingDispatcher( rpcService, @@ -191,7 +206,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), null, new MemoryArchivedExecutionGraphStore(), - new TestingJobManagerRunnerFactory(new CompletableFuture<>(), resultFuture, terminationFuture, failJobMasterCreationWith), + jobManagerRunnerFactory, fatalErrorHandler); dispatcher.start(); @@ -199,11 +214,6 @@ public class DispatcherResourceCleanupTest extends TestLogger { dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - - assertThat(blobFile.exists(), is(true)); - - // verify that we stored the blob also in the BlobStore - assertThat(storedHABlobFuture.get(), equalTo(permanentBlobKey)); } @After @@ -226,10 +236,10 @@ public class DispatcherResourceCleanupTest extends TestLogger { @Test public void testBlobServerCleanupWhenJobFinished() throws Exception { - submitJob(); + final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(); // complete the job - finishJob(); + finishJob(jobManagerRunnerFactory.takeCreatedJobManagerRunner()); assertThatHABlobsHaveBeenRemoved(); } @@ -250,11 +260,11 @@ public class DispatcherResourceCleanupTest extends TestLogger { @Test public void testBlobServerCleanupWhenJobNotFinished() throws Exception { - submitJob(); + final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(); // job not finished - resultFuture.completeExceptionally(new JobNotFinishedException(jobId)); - terminationFuture.complete(null); + final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + testingJobManagerRunner.completeResultFutureExceptionally(new JobNotFinishedException(jobId)); assertThat(cleanupJobFuture.get(), equalTo(jobId)); @@ -276,7 +286,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { */ @Test public void testBlobServerCleanupWhenJobSubmissionFails() throws Exception { - failJobMasterCreationWith.set(() -> new FlinkException("Test exception.")); + startDispatcher(new FailingJobManagerRunnerFactory(new FlinkException("Test exception"))); final CompletableFuture submissionFuture = dispatcherGateway.submitJob(jobGraph, timeout); try { @@ -291,11 +301,9 @@ public class DispatcherResourceCleanupTest extends TestLogger { @Test public void testBlobServerCleanupWhenClosingDispatcher() throws Exception { - submitJob(); + startDispatcherAndSubmitJob(); - dispatcher.closeAsync(); - terminationFuture.complete(null); - dispatcher.getTerminationFuture().get(); + dispatcher.closeAsync().get(); assertThat(cleanupJobFuture.get(), equalTo(jobId)); @@ -318,13 +326,13 @@ public class DispatcherResourceCleanupTest extends TestLogger { */ @Test public void testRunningJobsRegistryCleanup() throws Exception { - submitJob(); + final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(); runningJobsRegistry.setJobRunning(jobId); assertThat(runningJobsRegistry.contains(jobId), is(true)); - resultFuture.complete(new ArchivedExecutionGraphBuilder().setState(JobStatus.FINISHED).setJobID(jobId).build()); - terminationFuture.complete(null); + final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + testingJobManagerRunner.completeResultFuture(new ArchivedExecutionGraphBuilder().setState(JobStatus.FINISHED).setJobID(jobId).build()); // wait for the clearing clearedJobLatch.await(); @@ -338,10 +346,11 @@ public class DispatcherResourceCleanupTest extends TestLogger { */ @Test public void testJobSubmissionUnderSameJobId() throws Exception { - submitJob(); + final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(1); runningJobsRegistry.setJobRunning(jobId); - resultFuture.completeExceptionally(new JobNotFinishedException(jobId)); + final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + testingJobManagerRunner.completeResultFutureExceptionally(new JobNotFinishedException(jobId)); final CompletableFuture submissionFuture = dispatcherGateway.submitJob(jobGraph, timeout); @@ -352,7 +361,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { } catch (TimeoutException ignored) { // expected } finally { - terminationFuture.complete(null); + testingJobManagerRunner.completeTerminationFuture(); } assertThat(submissionFuture.get(), equalTo(Acknowledge.get())); @@ -364,7 +373,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { */ @Test public void testDuplicateJobSubmissionDoesNotDeleteJobMetaData() throws Exception { - submitJob(); + final TestingJobManagerRunnerFactoryNG testingJobManagerRunnerFactoryNG = startDispatcherAndSubmitJob(); final CompletableFuture submissionFuture = dispatcherGateway.submitJob(jobGraph, timeout); @@ -378,15 +387,14 @@ public class DispatcherResourceCleanupTest extends TestLogger { assertThatHABlobsHaveNotBeenRemoved(); } finally { - finishJob(); + finishJob(testingJobManagerRunnerFactoryNG.takeCreatedJobManagerRunner()); } assertThatHABlobsHaveBeenRemoved(); } - private void finishJob() { - resultFuture.complete(new ArchivedExecutionGraphBuilder().setJobID(jobId).setState(JobStatus.FINISHED).build()); - terminationFuture.complete(null); + private void finishJob(TestingJobManagerRunner takeCreatedJobManagerRunner) { + takeCreatedJobManagerRunner.completeResultFuture(new ArchivedExecutionGraphBuilder().setJobID(jobId).setState(JobStatus.FINISHED).build()); } private void assertThatHABlobsHaveNotBeenRemoved() { @@ -401,7 +409,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { */ @Test public void testJobRecoveryWithPendingTermination() throws Exception { - submitJob(); + final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(1); runningJobsRegistry.setJobRunning(jobId); dispatcherLeaderElectionService.notLeader(); @@ -414,7 +422,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { } catch (TimeoutException ignored) { // expected } finally { - terminationFuture.complete(null); + jobManagerRunnerFactory.takeCreatedJobManagerRunner().completeTerminationFuture(); } assertThat(leaderFuture.get(), equalTo(leaderSessionId)); @@ -477,15 +485,15 @@ public class DispatcherResourceCleanupTest extends TestLogger { @Test public void testHABlobsAreNotRemovedIfHAJobGraphRemovalFails() throws Exception { submittedJobGraphStore.setRemovalFailure(new Exception("Failed to Remove future")); - submitJob(); + final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(); ArchivedExecutionGraph executionGraph = new ArchivedExecutionGraphBuilder() .setJobID(jobId) .setState(JobStatus.CANCELED) .build(); - resultFuture.complete(executionGraph); - terminationFuture.complete(null); + final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + testingJobManagerRunner.completeResultFuture(executionGraph); assertThat(cleanupJobFuture.get(), equalTo(jobId)); assertThat(deleteAllHABlobsFuture.isDone(), is(false)); @@ -493,15 +501,15 @@ public class DispatcherResourceCleanupTest extends TestLogger { @Test public void testHABlobsAreRemovedIfHAJobGraphRemovalSucceeds() throws Exception { - submitJob(); + final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(); ArchivedExecutionGraph executionGraph = new ArchivedExecutionGraphBuilder() .setJobID(jobId) .setState(JobStatus.CANCELED) .build(); - resultFuture.complete(executionGraph); - terminationFuture.complete(null); + final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + testingJobManagerRunner.completeResultFuture(executionGraph); assertThat(cleanupJobFuture.get(), equalTo(jobId)); assertThat(deleteAllHABlobsFuture.get(), equalTo(jobId)); @@ -532,4 +540,17 @@ public class DispatcherResourceCleanupTest extends TestLogger { return result; } } + + private class FailingJobManagerRunnerFactory implements JobManagerRunnerFactory { + private final Exception testException; + + public FailingJobManagerRunnerFactory(FlinkException testException) { + this.testException = testException; + } + + @Override + public JobManagerRunner createJobManagerRunner(JobGraph jobGraph, Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, JobManagerSharedServices jobManagerServices, JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, FatalErrorHandler fatalErrorHandler) throws Exception { + throw testException; + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index daac79a311..69b6975474 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -44,6 +44,7 @@ import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobNotFinishedException; import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; @@ -774,7 +775,7 @@ public class DispatcherTest extends TestLogger { } @Override - public JobManagerRunner createJobManagerRunner(JobGraph jobGraph, Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, JobManagerSharedServices jobManagerSharedServices, JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, FatalErrorHandler fatalErrorHandler) throws Exception { + public TestingJobManagerRunner createJobManagerRunner(JobGraph jobGraph, Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, JobManagerSharedServices jobManagerSharedServices, JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, FatalErrorHandler fatalErrorHandler) throws Exception { jobManagerRunnerCreationLatch.run(); return super.createJobManagerRunner(jobGraph, configuration, rpcService, highAvailabilityServices, heartbeatServices, jobManagerSharedServices, jobManagerJobMetricGroupFactory, fatalErrorHandler); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java index 4e7c70fe7e..dae5c3ca70 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java @@ -22,13 +22,14 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import javax.annotation.Nonnull; + import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; @@ -39,8 +40,18 @@ public class TestingJobManagerRunnerFactoryNG implements JobManagerRunnerFactory private final BlockingQueue createdJobManagerRunner = new ArrayBlockingQueue<>(16); + private int numBlockingJobManagerRunners; + + public TestingJobManagerRunnerFactoryNG() { + this(0); + } + + public TestingJobManagerRunnerFactoryNG(int numBlockingJobManagerRunners) { + this.numBlockingJobManagerRunners = numBlockingJobManagerRunners; + } + @Override - public JobManagerRunner createJobManagerRunner( + public TestingJobManagerRunner createJobManagerRunner( JobGraph jobGraph, Configuration configuration, RpcService rpcService, @@ -49,12 +60,26 @@ public class TestingJobManagerRunnerFactoryNG implements JobManagerRunnerFactory JobManagerSharedServices jobManagerServices, JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, FatalErrorHandler fatalErrorHandler) throws Exception { - final TestingJobManagerRunner testingJobManagerRunner = new TestingJobManagerRunner(jobGraph.getJobID()); + final TestingJobManagerRunner testingJobManagerRunner = createTestingJobManagerRunner(jobGraph); createdJobManagerRunner.offer(testingJobManagerRunner); return testingJobManagerRunner; } + @Nonnull + private TestingJobManagerRunner createTestingJobManagerRunner(JobGraph jobGraph) { + final boolean blockingTermination; + + if (numBlockingJobManagerRunners > 0) { + numBlockingJobManagerRunners--; + blockingTermination = true; + } else { + blockingTermination = false; + } + + return new TestingJobManagerRunner(jobGraph.getJobID(), blockingTermination); + } + public TestingJobManagerRunner takeCreatedJobManagerRunner() throws InterruptedException { return createdJobManagerRunner.take(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java index fc29f0784a..b0a0b9c540 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java @@ -30,6 +30,8 @@ public class TestingJobManagerRunner implements JobManagerRunner { private final JobID jobId; + private final boolean blockingTermination; + private final CompletableFuture resultFuture; private final CompletableFuture jobMasterGatewayFuture; @@ -37,7 +39,12 @@ public class TestingJobManagerRunner implements JobManagerRunner { private final CompletableFuture terminationFuture; public TestingJobManagerRunner(JobID jobId) { + this(jobId, false); + } + + public TestingJobManagerRunner(JobID jobId, boolean blockingTermination) { this.jobId = jobId; + this.blockingTermination = blockingTermination; this.resultFuture = new CompletableFuture<>(); this.jobMasterGatewayFuture = new CompletableFuture<>(); this.terminationFuture = new CompletableFuture<>(); @@ -65,7 +72,9 @@ public class TestingJobManagerRunner implements JobManagerRunner { @Override public CompletableFuture closeAsync() { - terminationFuture.complete(null); + if (!blockingTermination) { + terminationFuture.complete(null); + } return terminationFuture; } @@ -73,4 +82,12 @@ public class TestingJobManagerRunner implements JobManagerRunner { public void completeResultFuture(ArchivedExecutionGraph archivedExecutionGraph) { resultFuture.complete(archivedExecutionGraph); } + + public void completeResultFutureExceptionally(Exception e) { + resultFuture.completeExceptionally(e); + } + + public void completeTerminationFuture() { + terminationFuture.complete(null); + } } -- Gitee From d4f80bf2f64937b7f233412c62fda6fd264fa239 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 7 May 2019 16:30:20 +0200 Subject: [PATCH 104/268] [FLINK-14260] Replace TestingJobManagerRunnerFactory with TestingJobManagerRunnerFactoryNG This commit removes the old TestingJobManagerRunnerFactory and renames the TestingJobManagerRunnerFactoryNG into TestingJobManagerRunnerFactory. This closes #9789. --- .../runtime/dispatcher/DispatcherHATest.java | 4 +- .../DispatcherResourceCleanupTest.java | 22 ++--- .../runtime/dispatcher/DispatcherTest.java | 2 +- .../dispatcher/MiniDispatcherTest.java | 4 +- .../TestingJobManagerRunnerFactory.java | 72 +++++++--------- .../TestingJobManagerRunnerFactoryNG.java | 86 ------------------- .../dispatcher/ZooKeeperHADispatcherTest.java | 8 +- 7 files changed, 52 insertions(+), 146 deletions(-) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.java index 713a6a2f11..172a46bfd4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.java @@ -291,8 +291,8 @@ public class DispatcherHATest extends TestLogger { } @Nonnull - private TestingJobManagerRunnerFactoryNG createTestingJobManagerRunnerFactory() { - return new TestingJobManagerRunnerFactoryNG(); + private TestingJobManagerRunnerFactory createTestingJobManagerRunnerFactory() { + return new TestingJobManagerRunnerFactory(); } private HATestingDispatcher createDispatcher(HighAvailabilityServices haServices) throws Exception { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 544137c803..33b88935b1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -181,12 +181,12 @@ public class DispatcherResourceCleanupTest extends TestLogger { fatalErrorHandler = new TestingFatalErrorHandler(); } - private TestingJobManagerRunnerFactoryNG startDispatcherAndSubmitJob() throws Exception { + private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob() throws Exception { return startDispatcherAndSubmitJob(0); } - private TestingJobManagerRunnerFactoryNG startDispatcherAndSubmitJob(int numBlockingJobManagerRunners) throws Exception { - final TestingJobManagerRunnerFactoryNG testingJobManagerRunnerFactoryNG = new TestingJobManagerRunnerFactoryNG(numBlockingJobManagerRunners); + private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob(int numBlockingJobManagerRunners) throws Exception { + final TestingJobManagerRunnerFactory testingJobManagerRunnerFactoryNG = new TestingJobManagerRunnerFactory(numBlockingJobManagerRunners); startDispatcher(testingJobManagerRunnerFactoryNG); submitJob(); @@ -236,7 +236,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { @Test public void testBlobServerCleanupWhenJobFinished() throws Exception { - final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(); + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(); // complete the job finishJob(jobManagerRunnerFactory.takeCreatedJobManagerRunner()); @@ -260,7 +260,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { @Test public void testBlobServerCleanupWhenJobNotFinished() throws Exception { - final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(); + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(); // job not finished final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); @@ -326,7 +326,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { */ @Test public void testRunningJobsRegistryCleanup() throws Exception { - final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(); + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(); runningJobsRegistry.setJobRunning(jobId); assertThat(runningJobsRegistry.contains(jobId), is(true)); @@ -346,7 +346,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { */ @Test public void testJobSubmissionUnderSameJobId() throws Exception { - final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(1); + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(1); runningJobsRegistry.setJobRunning(jobId); final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); @@ -373,7 +373,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { */ @Test public void testDuplicateJobSubmissionDoesNotDeleteJobMetaData() throws Exception { - final TestingJobManagerRunnerFactoryNG testingJobManagerRunnerFactoryNG = startDispatcherAndSubmitJob(); + final TestingJobManagerRunnerFactory testingJobManagerRunnerFactoryNG = startDispatcherAndSubmitJob(); final CompletableFuture submissionFuture = dispatcherGateway.submitJob(jobGraph, timeout); @@ -409,7 +409,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { */ @Test public void testJobRecoveryWithPendingTermination() throws Exception { - final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(1); + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(1); runningJobsRegistry.setJobRunning(jobId); dispatcherLeaderElectionService.notLeader(); @@ -485,7 +485,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { @Test public void testHABlobsAreNotRemovedIfHAJobGraphRemovalFails() throws Exception { submittedJobGraphStore.setRemovalFailure(new Exception("Failed to Remove future")); - final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(); + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(); ArchivedExecutionGraph executionGraph = new ArchivedExecutionGraphBuilder() .setJobID(jobId) @@ -501,7 +501,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { @Test public void testHABlobsAreRemovedIfHAJobGraphRemovalSucceeds() throws Exception { - final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = startDispatcherAndSubmitJob(); + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(); ArchivedExecutionGraph executionGraph = new ArchivedExecutionGraphBuilder() .setJobID(jobId) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index 69b6975474..c0ce6bc116 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -765,7 +765,7 @@ public class DispatcherTest extends TestLogger { dispatcher.getTerminationFuture().get(); } - private final class BlockingJobManagerRunnerFactory extends TestingJobManagerRunnerFactoryNG { + private final class BlockingJobManagerRunnerFactory extends TestingJobManagerRunnerFactory { @Nonnull private final ThrowingRunnable jobManagerRunnerCreationLatch; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java index 7630778a4a..bc2a239da2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java @@ -92,7 +92,7 @@ public class MiniDispatcherTest extends TestLogger { private TestingFatalErrorHandler testingFatalErrorHandler; - private TestingJobManagerRunnerFactoryNG testingJobManagerRunnerFactory; + private TestingJobManagerRunnerFactory testingJobManagerRunnerFactory; @BeforeClass public static void setupClass() throws IOException { @@ -119,7 +119,7 @@ public class MiniDispatcherTest extends TestLogger { highAvailabilityServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService); - testingJobManagerRunnerFactory = new TestingJobManagerRunnerFactoryNG(); + testingJobManagerRunnerFactory = new TestingJobManagerRunnerFactory(); } @After diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java index ea2580d021..c6f34d7264 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java @@ -19,76 +19,68 @@ package org.apache.flink.runtime.dispatcher; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmaster.JobManagerRunner; -import org.apache.flink.runtime.jobmaster.JobManagerRunnerImpl; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; +import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Supplier; +import javax.annotation.Nonnull; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; /** - * {@link JobManagerRunnerFactory} implementation for - * testing purposes. + * Testing implementation of {@link JobManagerRunnerFactory} which returns a {@link TestingJobManagerRunner}. */ class TestingJobManagerRunnerFactory implements JobManagerRunnerFactory { - private final CompletableFuture jobGraphFuture; - private final CompletableFuture resultFuture; - private final CompletableFuture terminationFuture; - private final AtomicReference> failJobMasterCreationWith; + private final BlockingQueue createdJobManagerRunner = new ArrayBlockingQueue<>(16); - TestingJobManagerRunnerFactory( - CompletableFuture jobGraphFuture, - CompletableFuture resultFuture, - CompletableFuture terminationFuture) { - this(jobGraphFuture, resultFuture, terminationFuture, new AtomicReference<>()); + private int numBlockingJobManagerRunners; + + public TestingJobManagerRunnerFactory() { + this(0); } - TestingJobManagerRunnerFactory( - CompletableFuture jobGraphFuture, - CompletableFuture resultFuture, - CompletableFuture terminationFuture, - AtomicReference> failJobMasterCreationWith) { - this.jobGraphFuture = jobGraphFuture; - this.resultFuture = resultFuture; - this.terminationFuture = terminationFuture; - this.failJobMasterCreationWith = failJobMasterCreationWith; + public TestingJobManagerRunnerFactory(int numBlockingJobManagerRunners) { + this.numBlockingJobManagerRunners = numBlockingJobManagerRunners; } @Override - public JobManagerRunner createJobManagerRunner( + public TestingJobManagerRunner createJobManagerRunner( JobGraph jobGraph, Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, - JobManagerSharedServices jobManagerSharedServices, + JobManagerSharedServices jobManagerServices, JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, FatalErrorHandler fatalErrorHandler) throws Exception { - final Supplier exceptionSupplier = failJobMasterCreationWith.get(); + final TestingJobManagerRunner testingJobManagerRunner = createTestingJobManagerRunner(jobGraph); + createdJobManagerRunner.offer(testingJobManagerRunner); - if (exceptionSupplier != null) { - throw exceptionSupplier.get(); - } else { - jobGraphFuture.complete(jobGraph); + return testingJobManagerRunner; + } - final JobManagerRunnerImpl mock = mock(JobManagerRunnerImpl.class); - when(mock.getResultFuture()).thenReturn(resultFuture); - when(mock.closeAsync()).thenReturn(terminationFuture); - when(mock.getJobID()).thenReturn(jobGraph.getJobID()); + @Nonnull + private TestingJobManagerRunner createTestingJobManagerRunner(JobGraph jobGraph) { + final boolean blockingTermination; - return mock; + if (numBlockingJobManagerRunners > 0) { + numBlockingJobManagerRunners--; + blockingTermination = true; + } else { + blockingTermination = false; } + + return new TestingJobManagerRunner(jobGraph.getJobID(), blockingTermination); + } + + public TestingJobManagerRunner takeCreatedJobManagerRunner() throws InterruptedException { + return createdJobManagerRunner.take(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java deleted file mode 100644 index dae5c3ca70..0000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactoryNG.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; -import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; -import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; -import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.runtime.rpc.RpcService; - -import javax.annotation.Nonnull; - -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; - -/** - * Testing implementation of {@link JobManagerRunnerFactory} which returns a {@link TestingJobManagerRunner}. - */ -public class TestingJobManagerRunnerFactoryNG implements JobManagerRunnerFactory { - - private final BlockingQueue createdJobManagerRunner = new ArrayBlockingQueue<>(16); - - private int numBlockingJobManagerRunners; - - public TestingJobManagerRunnerFactoryNG() { - this(0); - } - - public TestingJobManagerRunnerFactoryNG(int numBlockingJobManagerRunners) { - this.numBlockingJobManagerRunners = numBlockingJobManagerRunners; - } - - @Override - public TestingJobManagerRunner createJobManagerRunner( - JobGraph jobGraph, - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - HeartbeatServices heartbeatServices, - JobManagerSharedServices jobManagerServices, - JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, - FatalErrorHandler fatalErrorHandler) throws Exception { - final TestingJobManagerRunner testingJobManagerRunner = createTestingJobManagerRunner(jobGraph); - createdJobManagerRunner.offer(testingJobManagerRunner); - - return testingJobManagerRunner; - } - - @Nonnull - private TestingJobManagerRunner createTestingJobManagerRunner(JobGraph jobGraph) { - final boolean blockingTermination; - - if (numBlockingJobManagerRunners > 0) { - numBlockingJobManagerRunners--; - blockingTermination = true; - } else { - blockingTermination = false; - } - - return new TestingJobManagerRunner(jobGraph.getJobID(), blockingTermination); - } - - public TestingJobManagerRunner takeCreatedJobManagerRunner() throws InterruptedException { - return createdJobManagerRunner.take(); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java index 7c979692b0..bb29c258b3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java @@ -154,7 +154,7 @@ public class ZooKeeperHADispatcherTest extends TestLogger { final TestingDispatcher dispatcher = createDispatcher( testingHighAvailabilityServices, - new TestingJobManagerRunnerFactoryNG()); + new TestingJobManagerRunnerFactory()); dispatcher.start(); @@ -219,14 +219,14 @@ public class ZooKeeperHADispatcherTest extends TestLogger { final TestingLeaderElectionService leaderElectionService2 = new TestingLeaderElectionService(); haServices2.setDispatcherLeaderElectionService(leaderElectionService2); - final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = new TestingJobManagerRunnerFactoryNG(); + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = new TestingJobManagerRunnerFactory(); final TestingDispatcher dispatcher1 = createDispatcher( haServices1, jobManagerRunnerFactory); final TestingDispatcher dispatcher2 = createDispatcher( haServices2, - new TestingJobManagerRunnerFactoryNG()); + new TestingJobManagerRunnerFactory()); try { dispatcher1.start(); @@ -281,7 +281,7 @@ public class ZooKeeperHADispatcherTest extends TestLogger { try { haServices = new ZooKeeperHaServices(curatorFramework, rpcService.getExecutor(), configuration, new VoidBlobStore()); - final TestingJobManagerRunnerFactoryNG jobManagerRunnerFactory = new TestingJobManagerRunnerFactoryNG(); + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = new TestingJobManagerRunnerFactory(); dispatcher1 = createDispatcher( haServices, jobManagerRunnerFactory); -- Gitee From 7628d16aff75fd9ddc325e23f5f43c824258c1e5 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 17 Feb 2019 18:57:34 +0100 Subject: [PATCH 105/268] [FLINK-14261] Add PermanentlyFencedRpcEndpoint This closes #9790. --- .../flink/runtime/dispatcher/Dispatcher.java | 2 +- .../flink/runtime/jobmaster/JobMaster.java | 2 +- .../resourcemanager/ResourceManager.java | 2 +- .../flink/runtime/rpc/FencedRpcEndpoint.java | 10 ++-- .../rpc/PermanentlyFencedRpcEndpoint.java | 47 +++++++++++++++++++ .../flink/runtime/rpc/AsyncCallsTest.java | 11 +---- .../runtime/rpc/FencedRpcEndpointTest.java | 11 +---- 7 files changed, 57 insertions(+), 28 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/PermanentlyFencedRpcEndpoint.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index e3bfefb146..742e787f72 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -139,7 +139,7 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme String endpointId, DispatcherServices dispatcherServices, JobGraphStore jobGraphStore) throws Exception { - super(rpcService, endpointId); + super(rpcService, endpointId, null); Preconditions.checkNotNull(dispatcherServices); this.configuration = dispatcherServices.getConfiguration(); 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 ce949d8ae3..6b6b85cf8d 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 @@ -219,7 +219,7 @@ public class JobMaster extends FencedRpcEndpoint implements JobMast ShuffleMaster shuffleMaster, PartitionTrackerFactory partitionTrackerFactory) throws Exception { - super(rpcService, AkkaRpcServiceUtils.createRandomName(JOB_MANAGER_NAME)); + super(rpcService, AkkaRpcServiceUtils.createRandomName(JOB_MANAGER_NAME), null); this.jobMasterConfiguration = checkNotNull(jobMasterConfiguration); this.resourceId = checkNotNull(resourceId); 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 30ae329e4e..2295e3969a 100755 --- 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 @@ -168,7 +168,7 @@ public abstract class ResourceManager FatalErrorHandler fatalErrorHandler, JobManagerMetricGroup jobManagerMetricGroup) { - super(rpcService, resourceManagerEndpointId); + super(rpcService, resourceManagerEndpointId, null); this.resourceId = checkNotNull(resourceId); this.highAvailabilityServices = checkNotNull(highAvailabilityServices); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcEndpoint.java index 95d093194b..6dc9773de4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcEndpoint.java @@ -44,7 +44,7 @@ public abstract class FencedRpcEndpoint extends RpcEndpo private volatile F fencingToken; private volatile MainThreadExecutor fencedMainThreadExecutor; - protected FencedRpcEndpoint(RpcService rpcService, String endpointId) { + protected FencedRpcEndpoint(RpcService rpcService, String endpointId, @Nullable F fencingToken) { super(rpcService, endpointId); Preconditions.checkArgument( @@ -53,17 +53,17 @@ public abstract class FencedRpcEndpoint extends RpcEndpo FencedMainThreadExecutable.class.getSimpleName()); // no fencing token == no leadership - this.fencingToken = null; + this.fencingToken = fencingToken; this.unfencedMainThreadExecutor = new UnfencedMainThreadExecutor((FencedMainThreadExecutable) rpcServer); this.fencedMainThreadExecutor = new MainThreadExecutor( getRpcService().fenceRpcServer( rpcServer, - null), + fencingToken), this::validateRunsInMainThread); } - protected FencedRpcEndpoint(RpcService rpcService) { - this(rpcService, UUID.randomUUID().toString()); + protected FencedRpcEndpoint(RpcService rpcService, @Nullable F fencingToken) { + this(rpcService, UUID.randomUUID().toString(), fencingToken); } public F getFencingToken() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/PermanentlyFencedRpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/PermanentlyFencedRpcEndpoint.java new file mode 100644 index 0000000000..3da234f958 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/PermanentlyFencedRpcEndpoint.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.runtime.rpc; + +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +/** + * {@link FencedRpcEndpoint} which is fenced with a fencing token + * which is bound to the lifetime of the rpc endpoint. + * + * @param type of the fencing token. + */ +public class PermanentlyFencedRpcEndpoint extends FencedRpcEndpoint { + + protected PermanentlyFencedRpcEndpoint(RpcService rpcService, String endpointId, F fencingToken) { + super(rpcService, endpointId, Preconditions.checkNotNull(fencingToken)); + } + + protected PermanentlyFencedRpcEndpoint(RpcService rpcService, F fencingToken) { + super(rpcService, Preconditions.checkNotNull(fencingToken)); + } + + @Override + protected void setFencingToken(@Nullable F newFencingToken) { + throw new UnsupportedOperationException(String.format("Cannot change the fencing token of a %s.", PermanentlyFencedRpcEndpoint.class.getSimpleName())); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java index 5ebdce22a6..5d9745de76 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java @@ -414,22 +414,13 @@ public class AsyncCallsTest extends TestLogger { UUID initialFencingToken, OneShotLatch enteringSetNewFencingToken, OneShotLatch triggerSetNewFencingToken) { - super(rpcService); + super(rpcService, initialFencingToken); this.lock = lock; this.concurrentAccess = concurrentAccess; this.enteringSetNewFencingToken = enteringSetNewFencingToken; this.triggerSetNewFencingToken = triggerSetNewFencingToken; - - // make it look as if we are running in the main thread - currentMainThread.set(Thread.currentThread()); - - try { - setFencingToken(initialFencingToken); - } finally { - currentMainThread.set(null); - } } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java index ac350de823..43e5798574 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java @@ -296,20 +296,11 @@ public class FencedRpcEndpointTest extends TestLogger { } protected FencedTestingEndpoint(RpcService rpcService, String value, UUID initialFencingToken) { - super(rpcService); + super(rpcService, initialFencingToken); computationLatch = new OneShotLatch(); this.value = value; - - // make sure that it looks as if we are running in the main thread - currentMainThread.set(Thread.currentThread()); - - try { - setFencingToken(initialFencingToken); - } finally { - currentMainThread.set(null); - } } @Override -- Gitee From fda7ca929bb11fa598240468e51698b102ca3af2 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 15:57:32 +0200 Subject: [PATCH 106/268] [FLINK-14277][tests] Upgrade InMemoryJobGraphStore to TestingJobGraphStore Renames InMemoryJobGraphStore into TestingJobGraphStore and adds settable testing functions for all JobGraphStore methods. This changes makes the FaultyJobGraphStore obsolete and, hence, removes it. This closes #9806. --- .../runtime/dispatcher/DispatcherHATest.java | 15 +- .../DispatcherResourceCleanupTest.java | 17 ++- .../runtime/dispatcher/DispatcherTest.java | 82 ++++++----- .../dispatcher/FaultyJobGraphStore.java | 64 --------- ...phStore.java => TestingJobGraphStore.java} | 135 ++++++++++++++---- 5 files changed, 174 insertions(+), 139 deletions(-) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FaultyJobGraphStore.java rename flink-runtime/src/test/java/org/apache/flink/runtime/testutils/{InMemoryJobGraphStore.java => TestingJobGraphStore.java} (35%) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.java index 172a46bfd4..b25c60e7e6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.java @@ -42,7 +42,7 @@ 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.testtasks.NoOpInvokable; -import org.apache.flink.runtime.testutils.InMemoryJobGraphStore; +import org.apache.flink.runtime.testutils.TestingJobGraphStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.util.ExceptionUtils; @@ -207,13 +207,14 @@ public class DispatcherHATest extends TestLogger { */ @Test public void testJobRecoveryWhenChangingLeadership() throws Exception { - final InMemoryJobGraphStore submittedJobGraphStore = new InMemoryJobGraphStore(); - final CompletableFuture recoveredJobFuture = new CompletableFuture<>(); - submittedJobGraphStore.setRecoverJobGraphFunction((jobID, jobIDSubmittedJobGraphMap) -> { - recoveredJobFuture.complete(jobID); - return jobIDSubmittedJobGraphMap.get(jobID); - }); + final TestingJobGraphStore submittedJobGraphStore = TestingJobGraphStore.newBuilder() + .setRecoverJobGraphFunction( + (jobID, jobIDSubmittedJobGraphMap) -> { + recoveredJobFuture.complete(jobID); + return jobIDSubmittedJobGraphMap.get(jobID); + }) + .build(); final TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 33b88935b1..b0979a64fc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobNotFinishedException; @@ -53,6 +54,7 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.runtime.testutils.TestingJobGraphStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -128,7 +130,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { private CompletableFuture storedHABlobFuture; private CompletableFuture deleteAllHABlobsFuture; private CompletableFuture cleanupJobFuture; - private FaultyJobGraphStore submittedJobGraphStore; + private TestingJobGraphStore jobGraphStore; @BeforeClass public static void setupClass() { @@ -152,8 +154,8 @@ public class DispatcherResourceCleanupTest extends TestLogger { clearedJobLatch = new OneShotLatch(); runningJobsRegistry = new SingleRunningJobsRegistry(jobId, clearedJobLatch); highAvailabilityServices.setRunningJobsRegistry(runningJobsRegistry); - submittedJobGraphStore = new FaultyJobGraphStore(); - highAvailabilityServices.setJobGraphStore(submittedJobGraphStore); + jobGraphStore = TestingJobGraphStore.newBuilder().build(); + highAvailabilityServices.setJobGraphStore(jobGraphStore); storedHABlobFuture = new CompletableFuture<>(); deleteAllHABlobsFuture = new CompletableFuture<>(); @@ -484,7 +486,14 @@ public class DispatcherResourceCleanupTest extends TestLogger { @Test public void testHABlobsAreNotRemovedIfHAJobGraphRemovalFails() throws Exception { - submittedJobGraphStore.setRemovalFailure(new Exception("Failed to Remove future")); + final JobGraphStore jobGraphStore = TestingJobGraphStore.newBuilder() + .setRemoveJobGraphConsumer( + ignored -> { + throw new Exception("Failed to Remove future"); + }) + .build(); + + highAvailabilityServices.setJobGraphStore(jobGraphStore); final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(); ArchivedExecutionGraph executionGraph = new ArchivedExecutionGraphBuilder() diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index c0ce6bc116..44d0d04e51 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -63,7 +63,7 @@ import org.apache.flink.runtime.state.CheckpointStorageLocation; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.testtasks.NoOpInvokable; -import org.apache.flink.runtime.testutils.InMemoryJobGraphStore; +import org.apache.flink.runtime.testutils.TestingJobGraphStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -135,7 +135,7 @@ public class DispatcherTest extends TestLogger { private TestingFatalErrorHandler fatalErrorHandler; - private FaultyJobGraphStore submittedJobGraphStore; + private TestingJobGraphStore jobGraphStore; private TestingLeaderElectionService dispatcherLeaderElectionService; @@ -179,14 +179,14 @@ public class DispatcherTest extends TestLogger { fatalErrorHandler = new TestingFatalErrorHandler(); heartbeatServices = new HeartbeatServices(1000L, 10000L); - submittedJobGraphStore = new FaultyJobGraphStore(); + jobGraphStore = TestingJobGraphStore.newBuilder().build(); dispatcherLeaderElectionService = new TestingLeaderElectionService(); jobMasterLeaderElectionService = new TestingLeaderElectionService(); haServices = new TestingHighAvailabilityServices(); haServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService); - haServices.setJobGraphStore(submittedJobGraphStore); + haServices.setJobGraphStore(jobGraphStore); haServices.setJobMasterLeaderElectionService(TEST_JOB_ID, jobMasterLeaderElectionService); haServices.setCheckpointRecoveryFactory(new StandaloneCheckpointRecoveryFactory()); haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService()); @@ -314,14 +314,17 @@ public class DispatcherTest extends TestLogger { */ @Test public void testLeaderElection() throws Exception { - dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); - CompletableFuture jobIdsFuture = new CompletableFuture<>(); - submittedJobGraphStore.setJobIdsFunction( - (Collection jobIds) -> { - jobIdsFuture.complete(null); - return jobIds; - }); + final JobGraphStore jobGraphStore = TestingJobGraphStore.newBuilder() + .setJobIdsFunction( + (Collection jobIds) -> { + jobIdsFuture.complete(null); + return jobIds; + }) + .build(); + + haServices.setJobGraphStore(jobGraphStore); + dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); electDispatcher(); @@ -344,16 +347,16 @@ public class DispatcherTest extends TestLogger { dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); jobMasterLeaderElectionService.isLeader(UUID.randomUUID()).get(); - final JobGraph jobGraph = submittedJobGraphStore.recoverJobGraph(TEST_JOB_ID); + final JobGraph jobGraph = jobGraphStore.recoverJobGraph(TEST_JOB_ID); // pretend that other Dispatcher has removed job from submittedJobGraphStore - submittedJobGraphStore.removeJobGraph(TEST_JOB_ID); + jobGraphStore.removeJobGraph(TEST_JOB_ID); dispatcher.onRemovedJobGraph(TEST_JOB_ID); assertThat(dispatcherGateway.listJobs(TIMEOUT).get(), empty()); // pretend that other Dispatcher has added a job to submittedJobGraphStore runningJobsRegistry.clearJob(TEST_JOB_ID); - submittedJobGraphStore.putJobGraph(jobGraph); + jobGraphStore.putJobGraph(jobGraph); dispatcher.onAddedJobGraph(TEST_JOB_ID); createdJobManagerRunnerLatch.await(); assertThat(dispatcherGateway.listJobs(TIMEOUT).get(), hasSize(1)); @@ -361,14 +364,20 @@ public class DispatcherTest extends TestLogger { @Test public void testOnAddedJobGraphRecoveryFailure() throws Exception { - dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); - final FlinkException expectedFailure = new FlinkException("Expected failure"); - submittedJobGraphStore.setRecoveryFailure(expectedFailure); + final JobGraphStore jobGraphStore = TestingJobGraphStore.newBuilder() + .setRecoverJobGraphFunction( + (ignoredA, ignoredB) -> { + throw expectedFailure; + }) + .build(); + + haServices.setJobGraphStore(jobGraphStore); + dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - submittedJobGraphStore.putJobGraph(jobGraph); + jobGraphStore.putJobGraph(jobGraph); dispatcher.onAddedJobGraph(TEST_JOB_ID); final CompletableFuture errorFuture = fatalErrorHandler.getErrorFuture(); @@ -386,7 +395,7 @@ public class DispatcherTest extends TestLogger { dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - submittedJobGraphStore.putJobGraph(jobGraph); + jobGraphStore.putJobGraph(jobGraph); runningJobsRegistry.setJobFinished(TEST_JOB_ID); dispatcher.onAddedJobGraph(TEST_JOB_ID); @@ -460,7 +469,7 @@ public class DispatcherTest extends TestLogger { dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); // check that the job has been persisted - assertThat(submittedJobGraphStore.getJobIds(), contains(jobGraph.getJobID())); + assertThat(jobGraphStore.getJobIds(), contains(jobGraph.getJobID())); jobMasterLeaderElectionService.isLeader(UUID.randomUUID()).get(); @@ -557,13 +566,16 @@ public class DispatcherTest extends TestLogger { */ @Test public void testFatalErrorAfterJobIdRecoveryFailure() throws Exception { - dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); - final FlinkException testException = new FlinkException("Test exception"); - submittedJobGraphStore.setJobIdsFunction( - (Collection jobIds) -> { - throw testException; - }); + final JobGraphStore jobGraphStore = TestingJobGraphStore.newBuilder() + .setJobIdsFunction( + (Collection jobIds) -> { + throw testException; + }) + .build(); + + haServices.setJobGraphStore(jobGraphStore); + dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); electDispatcher(); @@ -582,17 +594,19 @@ public class DispatcherTest extends TestLogger { @Test public void testFatalErrorAfterJobRecoveryFailure() throws Exception { final FlinkException testException = new FlinkException("Test exception"); + final TestingJobGraphStore jobGraphStore = TestingJobGraphStore.newBuilder() + .setRecoverJobGraphFunction( + (JobID jobId, Map jobGraphs) -> { + throw testException; + }) + .build(); + haServices.setJobGraphStore(jobGraphStore); dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); dispatcher.waitUntilStarted(); - submittedJobGraphStore.putJobGraph(jobGraph); - - submittedJobGraphStore.setRecoverJobGraphFunction( - (JobID jobId, Map jobGraphs) -> { - throw testException; - }); + jobGraphStore.putJobGraph(jobGraph); electDispatcher(); @@ -618,7 +632,7 @@ public class DispatcherTest extends TestLogger { final JobGraph failingJobGraph = createFailingJobGraph(testException); - submittedJobGraphStore.putJobGraph(failingJobGraph); + jobGraphStore.putJobGraph(failingJobGraph); electDispatcher(); @@ -708,7 +722,7 @@ public class DispatcherTest extends TestLogger { @Test public void testPersistedJobGraphWhenDispatcherIsShutDown() throws Exception { - final InMemoryJobGraphStore submittedJobGraphStore = new InMemoryJobGraphStore(); + final TestingJobGraphStore submittedJobGraphStore = TestingJobGraphStore.newBuilder().build(); haServices.setJobGraphStore(submittedJobGraphStore); dispatcher = createAndStartDispatcher(heartbeatServices, haServices, DefaultJobManagerRunnerFactory.INSTANCE); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FaultyJobGraphStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FaultyJobGraphStore.java deleted file mode 100644 index 3ce028c929..0000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FaultyJobGraphStore.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.testutils.InMemoryJobGraphStore; - -import javax.annotation.Nullable; - -/** - * {@link InMemoryJobGraphStore} implementation which can throw artifical errors for - * testing purposes. - */ -final class FaultyJobGraphStore extends InMemoryJobGraphStore { - - @Nullable - private Exception recoveryFailure = null; - - @Nullable - private Exception removalFailure = null; - - void setRecoveryFailure(@Nullable Exception recoveryFailure) { - this.recoveryFailure = recoveryFailure; - } - - void setRemovalFailure(@Nullable Exception removalFailure) { - this.removalFailure = removalFailure; - } - - @Override - public synchronized JobGraph recoverJobGraph(JobID jobId) throws Exception { - if (recoveryFailure != null) { - throw recoveryFailure; - } else { - return super.recoverJobGraph(jobId); - } - } - - @Override - public synchronized void removeJobGraph(JobID jobId) throws Exception { - if (removalFailure != null) { - throw removalFailure; - } else { - super.removeJobGraph(jobId); - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/InMemoryJobGraphStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java similarity index 35% rename from flink-runtime/src/test/java/org/apache/flink/runtime/testutils/InMemoryJobGraphStore.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java index 31c3ea6eb8..bdf6f0b56c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/InMemoryJobGraphStore.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java @@ -24,6 +24,8 @@ import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.BiFunctionWithException; import org.apache.flink.util.function.FunctionWithException; +import org.apache.flink.util.function.ThrowingConsumer; +import org.apache.flink.util.function.ThrowingRunnable; import javax.annotation.Nullable; @@ -33,83 +35,88 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; -import static java.util.Objects.requireNonNull; - /** * In-Memory implementation of {@link JobGraphStore} for testing purposes. */ -public class InMemoryJobGraphStore implements JobGraphStore { +public class TestingJobGraphStore implements JobGraphStore { private final Map storedJobs = new HashMap<>(); - private boolean started; + private final ThrowingConsumer startConsumer; - private volatile FunctionWithException, Collection, ? extends Exception> jobIdsFunction; + private final ThrowingRunnable stopRunnable; - private volatile BiFunctionWithException, JobGraph, ? extends Exception> recoverJobGraphFunction; + private final FunctionWithException, Collection, ? extends Exception> jobIdsFunction; - public InMemoryJobGraphStore() { - jobIdsFunction = null; - recoverJobGraphFunction = null; - } + private final BiFunctionWithException, JobGraph, ? extends Exception> recoverJobGraphFunction; - public void setJobIdsFunction(FunctionWithException, Collection, ? extends Exception> jobIdsFunction) { - this.jobIdsFunction = Preconditions.checkNotNull(jobIdsFunction); - } + private final ThrowingConsumer putJobGraphConsumer; + + private final ThrowingConsumer removeJobGraphConsumer; - public void setRecoverJobGraphFunction(BiFunctionWithException, JobGraph, ? extends Exception> recoverJobGraphFunction) { - this.recoverJobGraphFunction = Preconditions.checkNotNull(recoverJobGraphFunction); + private final ThrowingConsumer releaseJobGraphConsumer; + + private boolean started; + + private TestingJobGraphStore( + ThrowingConsumer startConsumer, + ThrowingRunnable stopRunnable, + FunctionWithException, Collection, ? extends Exception> jobIdsFunction, + BiFunctionWithException, JobGraph, ? extends Exception> recoverJobGraphFunction, + ThrowingConsumer putJobGraphConsumer, + ThrowingConsumer removeJobGraphConsumer, + ThrowingConsumer releaseJobGraphConsumer) { + this.startConsumer = startConsumer; + this.stopRunnable = stopRunnable; + this.jobIdsFunction = jobIdsFunction; + this.recoverJobGraphFunction = recoverJobGraphFunction; + this.putJobGraphConsumer = putJobGraphConsumer; + this.removeJobGraphConsumer = removeJobGraphConsumer; + this.releaseJobGraphConsumer = releaseJobGraphConsumer; } @Override public synchronized void start(@Nullable JobGraphListener jobGraphListener) throws Exception { + startConsumer.accept(jobGraphListener); started = true; } @Override public synchronized void stop() throws Exception { + stopRunnable.run(); started = false; } @Override public synchronized JobGraph recoverJobGraph(JobID jobId) throws Exception { verifyIsStarted(); - - if (recoverJobGraphFunction != null) { - return recoverJobGraphFunction.apply(jobId, storedJobs); - } else { - return requireNonNull( - storedJobs.get(jobId), - "Job graph for job " + jobId + " does not exist"); - } + return recoverJobGraphFunction.apply(jobId, storedJobs); } @Override public synchronized void putJobGraph(JobGraph jobGraph) throws Exception { verifyIsStarted(); + putJobGraphConsumer.accept(jobGraph); storedJobs.put(jobGraph.getJobID(), jobGraph); } @Override public synchronized void removeJobGraph(JobID jobId) throws Exception { verifyIsStarted(); + removeJobGraphConsumer.accept(jobId); storedJobs.remove(jobId); } @Override - public void releaseJobGraph(JobID jobId) { + public synchronized void releaseJobGraph(JobID jobId) throws Exception { verifyIsStarted(); + releaseJobGraphConsumer.accept(jobId); } @Override public synchronized Collection getJobIds() throws Exception { verifyIsStarted(); - - if (jobIdsFunction != null) { - return jobIdsFunction.apply(storedJobs.keySet()); - } else { - return Collections.unmodifiableSet(new HashSet<>(storedJobs.keySet())); - } + return jobIdsFunction.apply(Collections.unmodifiableSet(new HashSet<>(storedJobs.keySet()))); } public synchronized boolean contains(JobID jobId) { @@ -120,4 +127,72 @@ public class InMemoryJobGraphStore implements JobGraphStore { Preconditions.checkState(started, "Not running. Forgot to call start()?"); } + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + private ThrowingConsumer startConsumer = ignored -> {}; + + private ThrowingRunnable stopRunnable = () -> {}; + + private FunctionWithException, Collection, ? extends Exception> jobIdsFunction = jobIds -> jobIds; + + private BiFunctionWithException, JobGraph, ? extends Exception> recoverJobGraphFunction = (jobId, jobs) -> jobs.get(jobId); + + private ThrowingConsumer putJobGraphConsumer = ignored -> {}; + + private ThrowingConsumer removeJobGraphConsumer = ignored -> {}; + + private ThrowingConsumer releaseJobGraphConsumer = ignored -> {}; + + private Builder() {} + + public Builder setStartConsumer(ThrowingConsumer startConsumer) { + this.startConsumer = startConsumer; + return this; + } + + public Builder setStopRunnable(ThrowingRunnable stopRunnable) { + this.stopRunnable = stopRunnable; + return this; + } + + public Builder setJobIdsFunction(FunctionWithException, Collection, ? extends Exception> jobIdsFunction) { + this.jobIdsFunction = jobIdsFunction; + return this; + } + + public Builder setRecoverJobGraphFunction(BiFunctionWithException, JobGraph, ? extends Exception> recoverJobGraphFunction) { + this.recoverJobGraphFunction = recoverJobGraphFunction; + return this; + } + + public Builder setPutJobGraphConsumer(ThrowingConsumer putJobGraphConsumer) { + this.putJobGraphConsumer = putJobGraphConsumer; + return this; + } + + public Builder setRemoveJobGraphConsumer(ThrowingConsumer removeJobGraphConsumer) { + this.removeJobGraphConsumer = removeJobGraphConsumer; + return this; + } + + public Builder setReleaseJobGraphConsumer(ThrowingConsumer releaseJobGraphConsumer) { + this.releaseJobGraphConsumer = releaseJobGraphConsumer; + return this; + } + + public TestingJobGraphStore build() { + return new TestingJobGraphStore( + startConsumer, + stopRunnable, + jobIdsFunction, + recoverJobGraphFunction, + putJobGraphConsumer, + removeJobGraphConsumer, + releaseJobGraphConsumer); + } + } + } -- Gitee From a7bf1ab1d5d47cb4dd69ac6a14d766b3857711eb Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 1 Oct 2019 14:54:36 +0200 Subject: [PATCH 107/268] [hotfix] Remove inheritance relationship between PartialDispatcherServices and DispatcherServices --- .../dispatcher/DispatcherServices.java | 103 ++++++++++++++++-- 1 file changed, 91 insertions(+), 12 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java index 0d7cc88014..bdb64638a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java @@ -33,7 +33,37 @@ import javax.annotation.Nullable; /** * {@link Dispatcher} services container. */ -public class DispatcherServices extends PartialDispatcherServices { +public class DispatcherServices { + + @Nonnull + private final Configuration configuration; + + @Nonnull + private final HighAvailabilityServices highAvailabilityServices; + + @Nonnull + private final GatewayRetriever resourceManagerGatewayRetriever; + + @Nonnull + private final BlobServer blobServer; + + @Nonnull + private final HeartbeatServices heartbeatServices; + + @Nonnull + private final JobManagerMetricGroup jobManagerMetricGroup; + + @Nonnull + private final ArchivedExecutionGraphStore archivedExecutionGraphStore; + + @Nonnull + private final FatalErrorHandler fatalErrorHandler; + + @Nonnull + private final HistoryServerArchivist historyServerArchivist; + + @Nullable + private final String metricQueryServiceAddress; @Nonnull private final JobManagerRunnerFactory jobManagerRunnerFactory; @@ -50,20 +80,69 @@ public class DispatcherServices extends PartialDispatcherServices { @Nonnull HistoryServerArchivist historyServerArchivist, @Nullable String metricQueryServiceAddress, @Nonnull JobManagerRunnerFactory jobManagerRunnerFactory) { - super( - configuration, - highAvailabilityServices, - resourceManagerGatewayRetriever, - blobServer, - heartbeatServices, - jobManagerMetricGroup, - archivedExecutionGraphStore, - fatalErrorHandler, - historyServerArchivist, - metricQueryServiceAddress); + this.configuration = configuration; + this.highAvailabilityServices = highAvailabilityServices; + this.resourceManagerGatewayRetriever = resourceManagerGatewayRetriever; + this.blobServer = blobServer; + this.heartbeatServices = heartbeatServices; + this.jobManagerMetricGroup = jobManagerMetricGroup; + this.archivedExecutionGraphStore = archivedExecutionGraphStore; + this.fatalErrorHandler = fatalErrorHandler; + this.historyServerArchivist = historyServerArchivist; + this.metricQueryServiceAddress = metricQueryServiceAddress; this.jobManagerRunnerFactory = jobManagerRunnerFactory; } + @Nonnull + public Configuration getConfiguration() { + return configuration; + } + + @Nonnull + public HighAvailabilityServices getHighAvailabilityServices() { + return highAvailabilityServices; + } + + @Nonnull + public GatewayRetriever getResourceManagerGatewayRetriever() { + return resourceManagerGatewayRetriever; + } + + @Nonnull + public BlobServer getBlobServer() { + return blobServer; + } + + @Nonnull + public HeartbeatServices getHeartbeatServices() { + return heartbeatServices; + } + + @Nonnull + public JobManagerMetricGroup getJobManagerMetricGroup() { + return jobManagerMetricGroup; + } + + @Nonnull + public ArchivedExecutionGraphStore getArchivedExecutionGraphStore() { + return archivedExecutionGraphStore; + } + + @Nonnull + public FatalErrorHandler getFatalErrorHandler() { + return fatalErrorHandler; + } + + @Nonnull + public HistoryServerArchivist getHistoryServerArchivist() { + return historyServerArchivist; + } + + @Nullable + public String getMetricQueryServiceAddress() { + return metricQueryServiceAddress; + } + @Nonnull JobManagerRunnerFactory getJobManagerRunnerFactory() { return jobManagerRunnerFactory; -- Gitee From d35f6990eb402a2137826b62bca11437728da9c1 Mon Sep 17 00:00:00 2001 From: Haibo Sun <7675577+sunhaibotb@users.noreply.github.com> Date: Wed, 2 Oct 2019 19:14:25 +0800 Subject: [PATCH 108/268] [FLINK-13516][test] Bump MiniKdc to 3.2.0 --- .../org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java | 3 --- pom.xml | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java index de15e49173..3faf777636 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java @@ -25,7 +25,6 @@ import org.apache.flink.runtime.security.SecurityUtils; import org.apache.flink.runtime.security.modules.HadoopModule; import org.apache.flink.test.util.SecureTestEnvironment; import org.apache.flink.test.util.TestingSecurityContext; -import org.apache.flink.testutils.junit.FailsOnJava11; import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; @@ -38,7 +37,6 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; -import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -108,7 +106,6 @@ public class YARNSessionFIFOSecuredITCase extends YARNSessionFIFOITCase { @Test(timeout = 60000) // timeout after a minute. @Override - @Category(FailsOnJava11.class) public void testDetachedMode() throws Exception { runTest(() -> { runDetachedModeTest(); diff --git a/pom.xml b/pom.xml index 73ac0aebbe..aafec9a3fa 100644 --- a/pom.xml +++ b/pom.xml @@ -140,7 +140,7 @@ under the License. Starting Hadoop 3, org.apache.kerby will be used instead of MiniKDC. We may have to revisit the impact at that time. --> - 2.7.2 + 3.2.0 ./docs/_includes/generated 2.3.4 JobClient. @@ -217,18 +214,19 @@ public class ClassLoaderITCase extends TestLogger { // Program should terminate with a 'SuccessException': // the exception class is contained in the user-jar, but is not present on the maven classpath // the deserialization of the exception should thus fail here + Optional exception = ExceptionUtils.findThrowable(e, + candidate -> candidate.getClass().getName().equals("org.apache.flink.test.classloading.jar.CheckpointedStreamingProgram$SuccessException")); + + if (!exception.isPresent()) { + // if this is achieved, either we failed due to another exception or the user-specific + // exception is not serialized between JobManager and JobClient. + throw e; + } + try { - Optional exception = ExceptionUtils.findThrowable(e, - candidate -> candidate.getClass().getCanonicalName().equals("org.apache.flink.test.classloading.jar.CheckpointedStreamingProgram.SuccessException")); - - // if we reach this point we either failed due to another exception, - // or the deserialization of the user-exception did not fail - if (!exception.isPresent()) { - throw e; - } else { - Assert.fail("Deserialization of user exception should have failed."); - } - } catch (NoClassDefFoundError expected) { + Class.forName(exception.get().getClass().getName()); + Assert.fail("Deserialization of user exception should have failed."); + } catch (ClassNotFoundException expected) { // expected } } -- Gitee From 2a182ffc1946fac7d2dcf7d16c408c53ef61466e Mon Sep 17 00:00:00 2001 From: sunhaibotb Date: Wed, 25 Sep 2019 10:20:10 +0800 Subject: [PATCH 114/268] [hotfix][test] Clean up unnecessary type argument declarations in ClassLoaderITCase --- .../test/classloading/ClassLoaderITCase.java | 49 +++++++++---------- 1 file changed, 23 insertions(+), 26 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java index e8aa408741..7ecf4887e8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java @@ -142,7 +142,7 @@ public class ClassLoaderITCase extends TestLogger { } @Test - public void testCustomSplitJobWithCustomClassLoaderJar() throws IOException, ProgramInvocationException { + public void testCustomSplitJobWithCustomClassLoaderJar() throws ProgramInvocationException { PackagedProgram inputSplitTestProg = new PackagedProgram(new File(INPUT_SPLITS_PROG_JAR_FILE)); @@ -150,20 +150,20 @@ public class ClassLoaderITCase extends TestLogger { miniClusterResource.getMiniCluster(), parallelism, Collections.singleton(new Path(INPUT_SPLITS_PROG_JAR_FILE)), - Collections.emptyList()); + Collections.emptyList()); inputSplitTestProg.invokeInteractiveModeForExecution(); } @Test - public void testStreamingCustomSplitJobWithCustomClassLoader() throws IOException, ProgramInvocationException { + public void testStreamingCustomSplitJobWithCustomClassLoader() throws ProgramInvocationException { PackagedProgram streamingInputSplitTestProg = new PackagedProgram(new File(STREAMING_INPUT_SPLITS_PROG_JAR_FILE)); TestStreamEnvironment.setAsContext( miniClusterResource.getMiniCluster(), parallelism, Collections.singleton(new Path(STREAMING_INPUT_SPLITS_PROG_JAR_FILE)), - Collections.emptyList()); + Collections.emptyList()); streamingInputSplitTestProg.invokeInteractiveModeForExecution(); } @@ -176,14 +176,14 @@ public class ClassLoaderITCase extends TestLogger { TestEnvironment.setAsContext( miniClusterResource.getMiniCluster(), parallelism, - Collections.emptyList(), + Collections.emptyList(), Collections.singleton(classpath)); inputSplitTestProg2.invokeInteractiveModeForExecution(); } @Test - public void testStreamingClassloaderJobWithCustomClassLoader() throws IOException, ProgramInvocationException { + public void testStreamingClassloaderJobWithCustomClassLoader() throws ProgramInvocationException { // regular streaming job PackagedProgram streamingProg = new PackagedProgram(new File(STREAMING_PROG_JAR_FILE)); @@ -191,13 +191,13 @@ public class ClassLoaderITCase extends TestLogger { miniClusterResource.getMiniCluster(), parallelism, Collections.singleton(new Path(STREAMING_PROG_JAR_FILE)), - Collections.emptyList()); + Collections.emptyList()); streamingProg.invokeInteractiveModeForExecution(); } @Test - public void testCheckpointedStreamingClassloaderJobWithCustomClassLoader() throws IOException, ProgramInvocationException { + public void testCheckpointedStreamingClassloaderJobWithCustomClassLoader() throws ProgramInvocationException { // checkpointed streaming job with custom classes for the checkpoint (FLINK-2543) // the test also ensures that user specific exceptions are serializable between JobManager <--> JobClient. PackagedProgram streamingCheckpointedProg = new PackagedProgram(new File(STREAMING_CHECKPOINTED_PROG_JAR_FILE)); @@ -206,7 +206,7 @@ public class ClassLoaderITCase extends TestLogger { miniClusterResource.getMiniCluster(), parallelism, Collections.singleton(new Path(STREAMING_CHECKPOINTED_PROG_JAR_FILE)), - Collections.emptyList()); + Collections.emptyList()); try { streamingCheckpointedProg.invokeInteractiveModeForExecution(); @@ -233,7 +233,7 @@ public class ClassLoaderITCase extends TestLogger { } @Test - public void testKMeansJobWithCustomClassLoader() throws IOException, ProgramInvocationException { + public void testKMeansJobWithCustomClassLoader() throws ProgramInvocationException { PackagedProgram kMeansProg = new PackagedProgram( new File(KMEANS_JAR_PATH), new String[] { @@ -246,20 +246,20 @@ public class ClassLoaderITCase extends TestLogger { miniClusterResource.getMiniCluster(), parallelism, Collections.singleton(new Path(KMEANS_JAR_PATH)), - Collections.emptyList()); + Collections.emptyList()); kMeansProg.invokeInteractiveModeForExecution(); } @Test - public void testUserCodeTypeJobWithCustomClassLoader() throws IOException, ProgramInvocationException { + public void testUserCodeTypeJobWithCustomClassLoader() throws ProgramInvocationException { PackagedProgram userCodeTypeProg = new PackagedProgram(new File(USERCODETYPE_JAR_PATH)); TestEnvironment.setAsContext( miniClusterResource.getMiniCluster(), parallelism, Collections.singleton(new Path(USERCODETYPE_JAR_PATH)), - Collections.emptyList()); + Collections.emptyList()); userCodeTypeProg.invokeInteractiveModeForExecution(); } @@ -280,10 +280,10 @@ public class ClassLoaderITCase extends TestLogger { miniClusterResource.getMiniCluster(), parallelism, Collections.singleton(new Path(CHECKPOINTING_CUSTOM_KV_STATE_JAR_PATH)), - Collections.emptyList()); + Collections.emptyList()); expectedException.expectCause( - Matchers.hasProperty("cause", isA(SuccessException.class))); + Matchers.hasProperty("cause", isA(SuccessException.class))); program.invokeInteractiveModeForExecution(); } @@ -313,20 +313,17 @@ public class ClassLoaderITCase extends TestLogger { miniClusterResource.getMiniCluster(), parallelism, Collections.singleton(new Path(CUSTOM_KV_STATE_JAR_PATH)), - Collections.emptyList() + Collections.emptyList() ); // Execute detached - Thread invokeThread = new Thread(new Runnable() { - @Override - public void run() { - try { - program.invokeInteractiveModeForExecution(); - } catch (ProgramInvocationException ignored) { - if (ignored.getCause() == null || - !(ignored.getCause() instanceof JobCancellationException)) { - ignored.printStackTrace(); - } + Thread invokeThread = new Thread(() -> { + try { + program.invokeInteractiveModeForExecution(); + } catch (ProgramInvocationException ex) { + if (ex.getCause() == null || + !(ex.getCause() instanceof JobCancellationException)) { + ex.printStackTrace(); } } }); -- Gitee From 3ef397deb75f6a58d73151fc88767e3932432d36 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Wed, 2 Oct 2019 15:06:07 +0200 Subject: [PATCH 115/268] [hotfix][runtime,tests] Fix checkstyle violations in ThrowableClassifierTest --- .../ThrowableClassifierTest.java | 31 ++++++++++--------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ThrowableClassifierTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ThrowableClassifierTest.java index 517321a5c3..b411c6d717 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ThrowableClassifierTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ThrowableClassifierTest.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.throwable.ThrowableAnnotation; import org.apache.flink.runtime.throwable.ThrowableClassifier; import org.apache.flink.runtime.throwable.ThrowableType; import org.apache.flink.util.TestLogger; + import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -31,8 +32,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; /** - * Test throwable classifier - * */ + * Test {@link ThrowableClassifier}. + */ public class ThrowableClassifierTest extends TestLogger { @Test @@ -49,25 +50,25 @@ public class ThrowableClassifierTest extends TestLogger { assertEquals(ThrowableType.RecoverableError, ThrowableClassifier.getThrowableType(new Exception(""))); assertEquals(ThrowableType.RecoverableError, - ThrowableClassifier.getThrowableType(new ThrowableType_RecoverableFailure_Exception())); + ThrowableClassifier.getThrowableType(new TestRecoverableErrorException())); } @Test public void testThrowableType_EnvironmentError() { assertEquals(ThrowableType.EnvironmentError, - ThrowableClassifier.getThrowableType(new ThrowableType_EnvironmentError_Exception())); + ThrowableClassifier.getThrowableType(new TestEnvironmentErrorException())); } @Test public void testThrowableType_PartitionDataMissingError() { assertEquals(ThrowableType.PartitionDataMissingError, - ThrowableClassifier.getThrowableType(new ThrowableType_PartitionDataMissingError_Exception())); + ThrowableClassifier.getThrowableType(new TestPartitionDataMissingErrorException())); } @Test public void testThrowableType_InheritError() { assertEquals(ThrowableType.PartitionDataMissingError, - ThrowableClassifier.getThrowableType(new Sub_ThrowableType_PartitionDataMissingError_Exception())); + ThrowableClassifier.getThrowableType(new TestPartitionDataMissingErrorSubException())); } @Test @@ -79,40 +80,40 @@ public class ThrowableClassifierTest extends TestLogger { // no recoverable throwable type assertFalse(ThrowableClassifier.findThrowableOfThrowableType( - new ThrowableType_PartitionDataMissingError_Exception(), + new TestPartitionDataMissingErrorException(), ThrowableType.RecoverableError).isPresent()); // direct recoverable throwable assertTrue(ThrowableClassifier.findThrowableOfThrowableType( - new ThrowableType_RecoverableFailure_Exception(), + new TestRecoverableErrorException(), ThrowableType.RecoverableError).isPresent()); // nested recoverable throwable assertTrue(ThrowableClassifier.findThrowableOfThrowableType( - new Exception(new ThrowableType_RecoverableFailure_Exception()), + new Exception(new TestRecoverableErrorException()), ThrowableType.RecoverableError).isPresent()); // inherit recoverable throwable assertTrue(ThrowableClassifier.findThrowableOfThrowableType( - new Sub_ThrowableType_RecoverableFailure_Exception(), + new TestRecoverableFailureSubException(), ThrowableType.RecoverableError).isPresent()); } @ThrowableAnnotation(ThrowableType.PartitionDataMissingError) - private class ThrowableType_PartitionDataMissingError_Exception extends Exception { + private class TestPartitionDataMissingErrorException extends Exception { } @ThrowableAnnotation(ThrowableType.EnvironmentError) - private class ThrowableType_EnvironmentError_Exception extends Exception { + private class TestEnvironmentErrorException extends Exception { } @ThrowableAnnotation(ThrowableType.RecoverableError) - private class ThrowableType_RecoverableFailure_Exception extends Exception { + private class TestRecoverableErrorException extends Exception { } - private class Sub_ThrowableType_PartitionDataMissingError_Exception extends ThrowableType_PartitionDataMissingError_Exception { + private class TestPartitionDataMissingErrorSubException extends TestPartitionDataMissingErrorException { } - private class Sub_ThrowableType_RecoverableFailure_Exception extends ThrowableType_RecoverableFailure_Exception { + private class TestRecoverableFailureSubException extends TestRecoverableErrorException { } } -- Gitee From 7ff6f1169ae4d5fcbff46ba3cd5d0fc47ef6bcc5 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Wed, 2 Oct 2019 14:55:19 +0200 Subject: [PATCH 116/268] [hotfix][runtime,tests] Move ThrowableClassifierTest to correct package --- .../ThrowableClassifierTest.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) rename flink-runtime/src/test/java/org/apache/flink/runtime/{executiongraph => throwable}/ThrowableClassifierTest.java (94%) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ThrowableClassifierTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/throwable/ThrowableClassifierTest.java similarity index 94% rename from flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ThrowableClassifierTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/throwable/ThrowableClassifierTest.java index b411c6d717..f6a8f190fd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ThrowableClassifierTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/throwable/ThrowableClassifierTest.java @@ -16,13 +16,10 @@ * limitations under the License. */ -package org.apache.flink.runtime.executiongraph; +package org.apache.flink.runtime.throwable; import org.apache.flink.runtime.execution.SuppressRestartsException; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; -import org.apache.flink.runtime.throwable.ThrowableAnnotation; -import org.apache.flink.runtime.throwable.ThrowableClassifier; -import org.apache.flink.runtime.throwable.ThrowableType; import org.apache.flink.util.TestLogger; import org.junit.Test; -- Gitee From 0d23d7b0bd2c5afb185c395f3da908b2f9177c2d Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Sat, 28 Sep 2019 14:45:58 +0800 Subject: [PATCH 117/268] [hotfix][runtime] Cleanup NoResourceAvailableException Remove unused methods and fix checkstyle violations. --- .../NoResourceAvailableException.java | 36 +++++-------------- 1 file changed, 9 insertions(+), 27 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java index bea7b8a06b..66008b4cd4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java @@ -22,37 +22,20 @@ import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.throwable.ThrowableAnnotation; import org.apache.flink.runtime.throwable.ThrowableType; +/** + * Indicates resource allocation failures. + */ @ThrowableAnnotation(ThrowableType.NonRecoverableError) public class NoResourceAvailableException extends JobException { private static final long serialVersionUID = -2249953165298717803L; - + private static final String BASE_MESSAGE = "Not enough free slots available to run the job. " - + "You can decrease the operator parallelism or increase the number of slots per TaskManager in the configuration."; + + "You can decrease the operator parallelism or increase the number of slots per TaskManager in the configuration."; public NoResourceAvailableException() { super(BASE_MESSAGE); } - - public NoResourceAvailableException(ScheduledUnit unit) { - super("No resource available to schedule unit " + unit - + ". You can decrease the operator parallelism or increase the number of slots per TaskManager in the configuration."); - } - - public NoResourceAvailableException(int numInstances, int numSlotsTotal, int availableSlots) { - super(String.format("%s Resources available to scheduler: Number of instances=%d, total number of slots=%d, available slots=%d", - BASE_MESSAGE, numInstances, numSlotsTotal, availableSlots)); - } - - NoResourceAvailableException(ScheduledUnit task, int numInstances, int numSlotsTotal, int availableSlots) { - super(String.format("%s Task to schedule: < %s > with groupID < %s > in sharing group < %s >. Resources available to scheduler: Number of instances=%d, total number of slots=%d, available slots=%d", - BASE_MESSAGE, task.getTaskToExecute(), - task.getCoLocationConstraint() == null ? task.getTaskToExecute().getVertex().getJobvertexId() : task.getCoLocationConstraint().getGroupId(), - task.getSlotSharingGroupId(), - numInstances, - numSlotsTotal, - availableSlots)); - } public NoResourceAvailableException(String message) { super(message); @@ -63,14 +46,13 @@ public class NoResourceAvailableException extends JobException { } // -------------------------------------------------------------------------------------------- - + @Override public boolean equals(Object obj) { - return obj instanceof NoResourceAvailableException && - getMessage().equals(((NoResourceAvailableException) obj).getMessage()); - + return obj instanceof NoResourceAvailableException && + getMessage().equals(((NoResourceAvailableException) obj).getMessage()); } - + @Override public int hashCode() { return getMessage().hashCode(); -- Gitee From 727b6c1cd0dc16b25b801b4c2ed71300ab9b97a5 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Sat, 28 Sep 2019 14:59:22 +0800 Subject: [PATCH 118/268] [hotifx][runtime] Remove NonRecoverableError annotation from NoResourceAvailableException With new generation scheduler, NonRecoverableError will suppress restarts and cause jobs to fail. Thus NoResourceAvailableException with NonRecoverableError annotation will lead jobs to be FAILED, which did not happen in the past. This should not happen because most issues that causes NoResourceAvailableException can be recovered via retries/restarts. To suppress restarts by purpose, one should pass a SuppressRestartsException as the param when creating a NoResourceAvailableException. --- .../jobmanager/scheduler/NoResourceAvailableException.java | 3 --- .../flink/runtime/throwable/ThrowableClassifierTest.java | 4 ---- 2 files changed, 7 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java index 66008b4cd4..905b224ced 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java @@ -19,13 +19,10 @@ package org.apache.flink.runtime.jobmanager.scheduler; import org.apache.flink.runtime.JobException; -import org.apache.flink.runtime.throwable.ThrowableAnnotation; -import org.apache.flink.runtime.throwable.ThrowableType; /** * Indicates resource allocation failures. */ -@ThrowableAnnotation(ThrowableType.NonRecoverableError) public class NoResourceAvailableException extends JobException { private static final long serialVersionUID = -2249953165298717803L; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/throwable/ThrowableClassifierTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/throwable/ThrowableClassifierTest.java index f6a8f190fd..08abd5e274 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/throwable/ThrowableClassifierTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/throwable/ThrowableClassifierTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.throwable; import org.apache.flink.runtime.execution.SuppressRestartsException; -import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -37,9 +36,6 @@ public class ThrowableClassifierTest extends TestLogger { public void testThrowableType_NonRecoverable() { assertEquals(ThrowableType.NonRecoverableError, ThrowableClassifier.getThrowableType(new SuppressRestartsException(new Exception("")))); - - assertEquals(ThrowableType.NonRecoverableError, - ThrowableClassifier.getThrowableType(new NoResourceAvailableException())); } @Test -- Gitee From b8a02d624c73ec4d30b30ba2aec3a0cc260b13f1 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Sat, 28 Sep 2019 16:28:23 +0800 Subject: [PATCH 119/268] [hotfix][runtime] Complete future returned by SimpleSlotProvider#allocateSlot with TimeoutException when the slot request cannot be fulfilled and queued scheduling is allowed This is similar to what the production SlotProvider does and can be helpful to verify some related error handlings. --- .../runtime/executiongraph/utils/SimpleSlotProvider.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java index f0db357aca..fe55bf3511 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java @@ -46,6 +46,7 @@ import java.net.InetAddress; import java.util.ArrayDeque; import java.util.HashMap; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeoutException; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -111,8 +112,9 @@ public class SimpleSlotProvider implements SlotProvider, SlotOwner { .createTestingLogicalSlot(); allocatedSlots.put(slotRequestId, slot); return CompletableFuture.completedFuture(result); - } - else { + } else if (allowQueued) { + return FutureUtils.completedExceptionally(new TimeoutException()); + } else { return FutureUtils.completedExceptionally(new NoResourceAvailableException()); } } -- Gitee From 013f8a14d3e6a46e1565c527dda96d90126425bd Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Sat, 28 Sep 2019 18:04:55 +0800 Subject: [PATCH 120/268] [FLINK-14247][runtime] Use NoResourceAvailableException to wrap TimeoutException on slot allocation timeout This closes #9794. --- .../flink/runtime/scheduler/DefaultScheduler.java | 15 ++++++++++++++- .../runtime/scheduler/DefaultSchedulerTest.java | 14 ++++++++++++++ 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java index 0d0af10609..c99949870c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.executiongraph.restart.ThrowingRestartStrategy; import org.apache.flink.runtime.io.network.partition.PartitionTracker; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; @@ -45,6 +46,7 @@ import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategy; import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategyFactory; import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.util.ExceptionUtils; import org.slf4j.Logger; @@ -59,6 +61,7 @@ import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; @@ -362,12 +365,22 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio .registerProducedPartitions(logicalSlot.getTaskManagerLocation(), sendScheduleOrUpdateConsumerMessage); executionVertex.tryAssignResource(logicalSlot); } else { - handleTaskFailure(executionVertexId, throwable); + handleTaskFailure(executionVertexId, maybeWrapWithNoResourceAvailableException(throwable)); } return null; }; } + private static Throwable maybeWrapWithNoResourceAvailableException(final Throwable failure) { + final Throwable strippedThrowable = ExceptionUtils.stripCompletionException(failure); + if (strippedThrowable instanceof TimeoutException) { + return new NoResourceAvailableException("Could not allocate the required slot within slot request timeout. " + + "Please make sure that the cluster has enough resources.", failure); + } else { + return failure; + } + } + private BiFunction deployOrHandleError(final DeploymentHandle deploymentHandle) { final ExecutionVertexVersion requiredVertexVersion = deploymentHandle.getRequiredVertexVersion(); final ExecutionVertexID executionVertexId = requiredVertexVersion.getExecutionVertexId(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java index 7988e73a97..a3e6e0e21c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java @@ -41,6 +41,7 @@ import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.ScheduleMode; +import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; import org.apache.flink.runtime.jobmaster.SlotRequestId; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; @@ -69,10 +70,13 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import static org.apache.flink.util.ExceptionUtils.findThrowable; +import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; /** * Tests for {@link DefaultScheduler}. @@ -238,6 +242,16 @@ public class DefaultSchedulerTest extends TestLogger { waitForTermination(scheduler); final JobStatus jobStatus = scheduler.requestJobStatus(); assertThat(jobStatus, is(Matchers.equalTo(JobStatus.FAILED))); + + Throwable failureCause = scheduler.requestJob() + .getFailureInfo() + .getException() + .deserializeError(DefaultSchedulerTest.class.getClassLoader()); + assertTrue(findThrowable(failureCause, NoResourceAvailableException.class).isPresent()); + assertTrue( + findThrowableWithMessage( + failureCause, + "Could not allocate the required slot within slot request timeout.").isPresent()); } private void drainAllAvailableSlots() { -- Gitee From 6de9c1e56238512b593d757270f20d00bddc687d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 1 Oct 2019 09:17:04 +0200 Subject: [PATCH 121/268] [hotfix] Reduce code duplication in MetricUtils --- .../runtime/metrics/util/MetricUtils.java | 19 +++++++++++-------- 1 file changed, 11 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 4ce8f5dd7d..13488b91ac 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 @@ -29,6 +29,7 @@ import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.rpc.RpcService; @@ -79,10 +80,7 @@ public class MetricUtils { metricRegistry, hostname); - MetricGroup statusGroup = jobManagerMetricGroup.addGroup(METRIC_GROUP_STATUS_NAME); - - // initialize the JM metrics - instantiateStatusMetrics(statusGroup); + createAndInitializeStatusMetricGroup(jobManagerMetricGroup); if (systemResourceProbeInterval.isPresent()) { instantiateSystemMetrics(jobManagerMetricGroup, systemResourceProbeInterval.get()); @@ -100,10 +98,7 @@ public class MetricUtils { hostName, resourceID.toString()); - MetricGroup statusGroup = taskManagerMetricGroup.addGroup(METRIC_GROUP_STATUS_NAME); - - // Initialize the TM metrics - instantiateStatusMetrics(statusGroup); + MetricGroup statusGroup = createAndInitializeStatusMetricGroup(taskManagerMetricGroup); if (systemResourceProbeInterval.isPresent()) { instantiateSystemMetrics(taskManagerMetricGroup, systemResourceProbeInterval.get()); @@ -111,6 +106,14 @@ public class MetricUtils { return Tuple2.of(taskManagerMetricGroup, statusGroup); } + private static MetricGroup createAndInitializeStatusMetricGroup(AbstractMetricGroup parentMetricGroup) { + MetricGroup statusGroup = parentMetricGroup.addGroup(METRIC_GROUP_STATUS_NAME); + + // Initialize the TM metrics + instantiateStatusMetrics(statusGroup); + return statusGroup; + } + public static void instantiateStatusMetrics( MetricGroup metricGroup) { MetricGroup jvm = metricGroup.addGroup("JVM"); -- Gitee From 972576954e9e3d8d6d6c5e23a6fcad9d60b0a6c8 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 1 Oct 2019 09:19:17 +0200 Subject: [PATCH 122/268] [FLINK-14299] Introduce ProcessMetricGroup The ProcessMetricGroup encapsulates the Status and system metrics of the ClusterEntrypoint. These have been factored out of the JobManagerMetricGroup. In order to maintain backwards compatibility, the same scope as the JobManagerMetricGroup is being used. --- .../runtime/entrypoint/ClusterEntrypoint.java | 16 +++++ ...atcherResourceManagerComponentFactory.java | 12 +--- .../metrics/groups/ProcessMetricGroup.java | 65 +++++++++++++++++++ .../groups/UnregisteredMetricGroups.java | 15 +++++ .../runtime/metrics/util/MetricUtils.java | 21 ++++-- .../runtime/minicluster/MiniCluster.java | 15 +++++ .../apache/flink/runtime/rpc/RpcUtils.java | 13 ++++ 7 files changed, 141 insertions(+), 16 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/ProcessMetricGroup.java 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 0538becf3b..7f48fa02a2 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 @@ -46,10 +46,12 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.ReporterSetup; +import org.apache.flink.runtime.metrics.groups.ProcessMetricGroup; import org.apache.flink.runtime.metrics.util.MetricUtils; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityContext; @@ -121,6 +123,9 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro @GuardedBy("lock") private MetricRegistryImpl metricRegistry; + @GuardedBy("lock") + private ProcessMetricGroup processMetricGroup; + @GuardedBy("lock") private HighAvailabilityServices haServices; @@ -263,6 +268,13 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro final RpcService metricQueryServiceRpcService = MetricUtils.startMetricsRpcService(configuration, bindAddress); metricRegistry.startQueryService(metricQueryServiceRpcService, null); + final String hostname = RpcUtils.getHostname(commonRpcService); + + processMetricGroup = MetricUtils.instantiateProcessMetricGroup( + metricRegistry, + hostname, + ConfigurationUtils.getSystemResourceMetricsProbingInterval(configuration)); + archivedExecutionGraphStore = createSerializableExecutionGraphStore(configuration, commonRpcService.getScheduledExecutor()); } } @@ -349,6 +361,10 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro } } + if (processMetricGroup != null) { + processMetricGroup.close(); + } + if (metricRegistry != null) { terminationFutures.add(metricRegistry.shutdown()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java index 6d50ed0490..6353f729b1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java @@ -20,7 +20,6 @@ package org.apache.flink.runtime.entrypoint.component; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.MetricOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.runtime.blob.BlobServer; @@ -50,6 +49,7 @@ import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcherImpl; import org.apache.flink.runtime.rest.handler.legacy.metrics.VoidMetricFetcher; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever; @@ -161,12 +161,11 @@ public abstract class AbstractDispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponent( T dispatcher, ResourceManager resourceManager, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/ProcessMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/ProcessMetricGroup.java new file mode 100644 index 0000000000..74ffddff71 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/ProcessMetricGroup.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.runtime.metrics.groups; + +import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; +import org.apache.flink.runtime.metrics.scope.ScopeFormat; + +import java.util.Map; + +/** + * {@link AbstractMetricGroup} implementation for process related metrics. + */ +public class ProcessMetricGroup extends AbstractMetricGroup> { + private final String hostname; + + ProcessMetricGroup(MetricRegistry registry, String hostname) { + super(registry, getScope(registry, hostname), null); + this.hostname = hostname; + } + + private static String[] getScope(MetricRegistry registry, String hostname) { + // returning jobmanager scope in order to guarantee backwards compatibility + // this can be changed once we introduce a proper scope for the process metric group + return registry.getScopeFormats().getJobManagerFormat().formatScope(hostname); + } + + @Override + protected String getGroupName(CharacterFilter filter) { + // returning jobmanager in order to guarantee backwards compatibility + // this can be changed once we introduce a proper group name for the process metric group + return "jobmanager"; + } + + @Override + protected void putVariables(Map variables) { + variables.put(ScopeFormat.SCOPE_HOST, hostname); + } + + @Override + protected QueryScopeInfo createQueryServiceMetricInfo(CharacterFilter filter) { + return new QueryScopeInfo.JobManagerQueryScopeInfo(); + } + + public static ProcessMetricGroup create(MetricRegistry metricRegistry, String hostname) { + return new ProcessMetricGroup(metricRegistry, hostname); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/UnregisteredMetricGroups.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/UnregisteredMetricGroups.java index 8c635eb994..48b7f5a1c7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/UnregisteredMetricGroups.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/UnregisteredMetricGroups.java @@ -33,6 +33,10 @@ public class UnregisteredMetricGroups { private UnregisteredMetricGroups() { } + public static ProcessMetricGroup createUnregisteredProcessMetricGroup() { + return new UnregisteredProcessMetricGroup(); + } + public static JobManagerMetricGroup createUnregisteredJobManagerMetricGroup() { return new UnregisteredJobManagerMetricGroup(); } @@ -57,6 +61,17 @@ public class UnregisteredMetricGroups { return new UnregisteredOperatorMetricGroup(); } + /** + * A safe drop-in replacement for {@link ProcessMetricGroup ProcessMetricGroups}. + */ + public static class UnregisteredProcessMetricGroup extends ProcessMetricGroup { + private static final String UNREGISTERED_HOST = "UnregisteredHost"; + + public UnregisteredProcessMetricGroup() { + super(NoOpMetricRegistry.INSTANCE, UNREGISTERED_HOST); + } + } + /** * A safe drop-in replacement for {@link JobManagerMetricGroup}s. */ 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 13488b91ac..cda453d469 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 @@ -31,6 +31,7 @@ import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; +import org.apache.flink.runtime.metrics.groups.ProcessMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; @@ -72,19 +73,26 @@ public class MetricUtils { private MetricUtils() { } - public static JobManagerMetricGroup instantiateJobManagerMetricGroup( + public static ProcessMetricGroup instantiateProcessMetricGroup( final MetricRegistry metricRegistry, final String hostname, final Optional

+Internally, an instance of the assigner is executed per shard / consumer thread (see threading model below). +When an assigner is specified, for each record read from Kinesis, the extractTimestamp(T element, long previousElementTimestamp) +is called to assign a timestamp to the record and getCurrentWatermark() to determine the new watermark for the shard. +The watermark of the consumer subtask is then determined as the minimum watermark of all its shards and emitted periodically. +The per shard watermark is essential to deal with varying consumption speed between shards, that otherwise could lead +to issues with downstream logic that relies on the watermark, such as incorrect late data dropping. + +By default, the watermark is going to stall if shards do not deliver new records. +The property `ConsumerConfigConstants.SHARD_IDLE_INTERVAL_MILLIS` can be used to avoid this potential issue through a +timeout that will allow the watermark to progress despite of idle shards. + +### Event Time Alignment for Shard Consumers + +The Flink Kinesis Consumer optionally supports synchronization between parallel consumer subtasks (and their threads) +to avoid the event time skew related problems described in [Event time synchronization across sources](https://issues.apache.org/jira/browse/FLINK-10886). + +To enable synchronization, set the watermark tracker on the consumer: + +
+{% highlight java %} +JobManagerWatermarkTracker watermarkTracker = + new JobManagerWatermarkTracker("myKinesisSource"); +consumer.setWatermarkTracker(watermarkTracker); +{% endhighlight %} +
+ +The `JobManagerWatermarkTracker` will use a global aggregate to synchronize the per subtask watermarks. Each subtask +uses a per shard queue to control the rate at which records are emitted downstream based on how far ahead of the global +watermark the next record in the queue is. + +The "emit ahead" limit is configured via `ConsumerConfigConstants.WATERMARK_LOOKAHEAD_MILLIS`. Smaller values reduce +the skew but also the throughput. Larger values will allow the subtask to proceed further before waiting for the global +watermark to advance. + +Another variable in the throughput equation is how frequently the watermark is propagated by the tracker. +The interval can be configured via `ConsumerConfigConstants.WATERMARK_SYNC_MILLIS`. +Smaller values reduce emitter waits and come at the cost of increased communication with the job manager. + +Since records accumulate in the queues when skew occurs, increased memory consumption needs to be expected. +How much depends on the average record size. With larger sizes, it may be necessary to adjust the emitter queue capacity via +`ConsumerConfigConstants.WATERMARK_SYNC_QUEUE_CAPACITY`. + ### Threading Model The Flink Kinesis Consumer uses multiple threads for shard discovery and data consumption. @@ -221,7 +273,7 @@ on the APIs, the consumer will be competing with other non-Flink consuming appli Below is a list of APIs called by the consumer with description of how the consumer uses the API, as well as information on how to deal with any errors or warnings that the Flink Kinesis Consumer may have due to these service limits. -- *[DescribeStream](http://docs.aws.amazon.com/kinesis/latest/APIReference/API_DescribeStream.html)*: this is constantly called +- *[ListShards](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ListShards.html)*: this is constantly called by a single thread in each parallel consumer subtask to discover any new shards as a result of stream resharding. By default, the consumer performs the shard discovery at an interval of 10 seconds, and will retry indefinitely until it gets a result from Kinesis. If this interferes with other non-Flink consuming applications, users can slow down the consumer of -- Gitee From fdfa03bd286a86b8d44a693f18772eedcff69501 Mon Sep 17 00:00:00 2001 From: Ricco Chen Date: Sat, 5 Oct 2019 22:57:22 +0800 Subject: [PATCH 131/268] [FLINK-14117][docs-zh] Translate changes on index page to Chinese (#9815) --- docs/index.zh.md | 28 +++++++++++++++++++++++----- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/docs/index.zh.md b/docs/index.zh.md index 898a2a2e03..94a39df01a 100644 --- a/docs/index.zh.md +++ b/docs/index.zh.md @@ -30,14 +30,32 @@ Apache Flink 是一个分布式流批一体化的开源平台。Flink 的核心 ## 初步印象 -- **概念**: 从 Flink 的 [数据流编程模型](concepts/programming-model.html) 和 [分布式执行环境](concepts/runtime.html) 开始了解最基本的概念。这能帮助你理解本文档的其他部分,包括如何搭建环境,进行程序编写等。建议你首先阅读此部分。 -- **教程**: - * [实现并运行一个 DataStream 作业](./getting-started/tutorials/datastream_api.html) - * [搭建一个本地 Flink 集群](./getting-started/tutorials/local_setup.html) +* **代码练习**: 跟随分步指南通过 Flink API 实现简单应用或查询。 + * [实现 DataStream 应用](./getting-started/walkthroughs/datastream_api.html) + * [书写 Table API 查询](./getting-started/walkthroughs/table_api.html) -- **编程指南**: 你可以从 [基本 API 概念](dev/api_concepts.html), [DataStream API](dev/datastream_api.html) 以及 [DataSet API](dev/batch/index.html) 着手学习如何编写你的第一个 Flink 作业。 +* **Docker 游乐场**: 你只需花几分钟搭建 Flink 沙盒环境,就可以探索和使用 Flink 了。 + * [运行与管理 Flink 流处理应用](./getting-started/docker-playgrounds/flink-operations-playground.html) + +* **教程**: 在你的本地机器上安装 Flink。 + * [搭建本地 Flink 集群](./getting-started/tutorials/local_setup.html) + +* **概念**: 学习 Flink 的基本概念能更好地理解文档。 + * [数据流编程模型](concepts/programming-model.html) + * [分布式执行](concepts/runtime.html) + * [术语表](concepts/glossary.html) + +## API 参考 + +API 参考列举并解释了 Flink API 的所有功能。 + +* [基本 API 概念](dev/api_concepts.html) +* [DataStream API](dev/datastream_api.html) +* [DataSet API](dev/batch/index.html) +* [Table API & SQL](dev/table/index.html) ## 部署 + 在线上环境运行你的 Flink 作业之前,请阅读 [生产环境注意事项检查清单](ops/production_ready.html)。 ## 发布日志 -- Gitee From 503ae365437e4cb7580defd7e8e26b72d5beeb18 Mon Sep 17 00:00:00 2001 From: yangjf2019 Date: Wed, 25 Sep 2019 10:40:56 +0800 Subject: [PATCH 132/268] [FLINK-13037][docs-zh] Translate "Concepts -> Glossary" page into Chinese This closes #9763 --- docs/concepts/glossary.zh.md | 100 +++++++++-------------------------- 1 file changed, 25 insertions(+), 75 deletions(-) diff --git a/docs/concepts/glossary.zh.md b/docs/concepts/glossary.zh.md index 22f2c99ef2..5c5bd1d4cb 100644 --- a/docs/concepts/glossary.zh.md +++ b/docs/concepts/glossary.zh.md @@ -1,7 +1,7 @@ --- -title: Glossary +title: 词汇表 nav-pos: 3 -nav-title: Glossary +nav-title: 词汇表 nav-parent_id: concepts --- (b1) -+-> (c1) ---+-> (d1) - * X / - * (a2) ---> (b2) -+-> (c2) -+ - * - * ^ ^ ^ - * | | | - * (pipelined) (blocking) (pipelined) - * - * - */ - @Test - public void testMultiRegionsFailover() throws Exception { - final JobID jobId = new JobID(); - final String jobName = "Test Job Sample Name"; - - final Map attemptIDInitStateMap = new HashMap<>(); - final SlotProvider slotProvider = new SimpleSlotProvider(jobId, 20, new CollectTddTaskManagerGateway(attemptIDInitStateMap)); - - JobVertex v1 = new JobVertex("vertex1"); - JobVertex v2 = new JobVertex("vertex2"); - JobVertex v3 = new JobVertex("vertex3"); - JobVertex v4 = new JobVertex("vertex4"); - - v1.setParallelism(2); - v2.setParallelism(2); - v3.setParallelism(2); - v4.setParallelism(1); - - v1.setInvokableClass(AbstractInvokable.class); - v2.setInvokableClass(AbstractInvokable.class); - v3.setInvokableClass(AbstractInvokable.class); - v4.setInvokableClass(AbstractInvokable.class); - - v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED); - v3.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - v4.connectNewDataSetAsInput(v3, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - List ordered = Arrays.asList(v1, v2, v3, v4); - - final ExecutionGraph eg = new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(jobId, jobName, v1, v2, v3, v4) - .setRestartStrategy(new InfiniteDelayRestartStrategy(10)) - .setFailoverStrategyFactory(new FailoverPipelinedRegionWithDirectExecutor()) - .setSlotProvider(slotProvider) - .allowQueuedScheduling() - .build(); - - RestartPipelinedRegionStrategy strategy = (RestartPipelinedRegionStrategy) eg.getFailoverStrategy(); - - // the following two vertices are in the same failover region - ExecutionVertex ev11 = eg.getJobVertex(v1.getID()).getTaskVertices()[0]; - ExecutionVertex ev21 = eg.getJobVertex(v2.getID()).getTaskVertices()[0]; - - // the following two vertices are in the same failover region - ExecutionVertex ev12 = eg.getJobVertex(v1.getID()).getTaskVertices()[1]; - ExecutionVertex ev22 = eg.getJobVertex(v2.getID()).getTaskVertices()[1]; - - // the following vertices are in one failover region - ExecutionVertex ev31 = eg.getJobVertex(v3.getID()).getTaskVertices()[0]; - ExecutionVertex ev32 = eg.getJobVertex(v3.getID()).getTaskVertices()[1]; - ExecutionVertex ev4 = eg.getJobVertex(v4.getID()).getTaskVertices()[0]; - - enableCheckpointing(eg); - - eg.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); - eg.scheduleForExecution(); - assertEquals(JobStatus.RUNNING, eg.getState()); - - attachPendingCheckpoints(eg); - - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev11).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev21).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev31).getState()); - - acknowledgeAllCheckpoints(eg.getCheckpointCoordinator(), Arrays.asList(ev11, ev21, ev12, ev22, ev31, ev32, ev4).iterator()); - - ev21.scheduleForExecution(eg.getSlotProviderStrategy(), LocationPreferenceConstraint.ALL, Collections.emptySet()); - ev21.getCurrentExecutionAttempt().fail(new Exception("New fail")); - assertEquals(JobStatus.CANCELLING, strategy.getFailoverRegion(ev11).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev22).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev31).getState()); - - ev11.getCurrentExecutionAttempt().completeCancelling(); - verifyCheckpointRestoredAsExpected(eg); - - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev11).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev22).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev31).getState()); - - ev11.getCurrentExecutionAttempt().markFinished(); - ev21.getCurrentExecutionAttempt().markFinished(); - ev22.scheduleForExecution(eg.getSlotProviderStrategy(), LocationPreferenceConstraint.ALL, Collections.emptySet()); - ev22.getCurrentExecutionAttempt().markFinished(); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev11).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev22).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev31).getState()); - - waitUntilExecutionState(ev31.getCurrentExecutionAttempt(), ExecutionState.DEPLOYING, 2000); - waitUntilExecutionState(ev32.getCurrentExecutionAttempt(), ExecutionState.DEPLOYING, 2000); - - ev31.getCurrentExecutionAttempt().fail(new Exception("New fail")); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev11).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev22).getState()); - assertEquals(JobStatus.CANCELLING, strategy.getFailoverRegion(ev31).getState()); - - ev32.getCurrentExecutionAttempt().completeCancelling(); - verifyCheckpointRestoredAsExpected(eg); - - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev11).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev22).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev31).getState()); - - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev4).getState()); - } - - /** - * Tests that when a task fail, and restart strategy doesn't support restarting, the job will go to failed. - * @throws Exception if fail to create the single region execution graph. - */ - @Test - public void testNoManualRestart() throws Exception { - NoRestartStrategy restartStrategy = new NoRestartStrategy(); - ExecutionGraph eg = createSingleRegionExecutionGraph(restartStrategy); - - ExecutionVertex ev = eg.getAllExecutionVertices().iterator().next(); - - ev.fail(new Exception("Test Exception")); - - for (ExecutionVertex evs : eg.getAllExecutionVertices()) { - evs.getCurrentExecutionAttempt().completeCancelling(); - } - assertEquals(JobStatus.FAILED, eg.getState()); - } - - /** - * Tests that two regions failover at the same time, they will not influence each other. - * @throws Exception if fail to create dummy job information, fail to schedule for execution - * or timeout before region switches to expected status. - */ - @Test - public void testMultiRegionFailoverAtSameTime() throws Exception { - final JobID jobId = new JobID(); - final String jobName = "Test Job Sample Name"; - - final SimpleSlotProvider slotProvider = new SimpleSlotProvider(jobId, 16); - - JobVertex v1 = new JobVertex("vertex1"); - JobVertex v2 = new JobVertex("vertex2"); - JobVertex v3 = new JobVertex("vertex3"); - JobVertex v4 = new JobVertex("vertex4"); - - v1.setParallelism(2); - v2.setParallelism(2); - v3.setParallelism(2); - v4.setParallelism(2); - - v1.setInvokableClass(AbstractInvokable.class); - v2.setInvokableClass(AbstractInvokable.class); - v3.setInvokableClass(AbstractInvokable.class); - v4.setInvokableClass(AbstractInvokable.class); - - v2.connectNewDataSetAsInput(v1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - v4.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - v4.connectNewDataSetAsInput(v3, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - List ordered = Arrays.asList(v1, v2, v3, v4); - - ExecutionGraph eg = new ExecutionGraph( - new DummyJobInformation( - jobId, - jobName), - TestingUtils.defaultExecutor(), - TestingUtils.defaultExecutor(), - AkkaUtils.getDefaultTimeout(), - new InfiniteDelayRestartStrategy(10), - new RestartPipelinedRegionStrategy.Factory(), - slotProvider); - try { - eg.attachJobGraph(ordered); - } - catch (JobException e) { - e.printStackTrace(); - fail("Job failed with exception: " + e.getMessage()); - } - eg.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); - eg.scheduleForExecution(); - RestartPipelinedRegionStrategy strategy = (RestartPipelinedRegionStrategy)eg.getFailoverStrategy(); - - ExecutionVertex ev11 = eg.getJobVertex(v1.getID()).getTaskVertices()[0]; - ExecutionVertex ev12 = eg.getJobVertex(v1.getID()).getTaskVertices()[1]; - ExecutionVertex ev31 = eg.getJobVertex(v3.getID()).getTaskVertices()[0]; - ExecutionVertex ev32 = eg.getJobVertex(v3.getID()).getTaskVertices()[1]; - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev11).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev31).getState()); - - ev11.getCurrentExecutionAttempt().fail(new Exception("new fail")); - ev31.getCurrentExecutionAttempt().fail(new Exception("new fail")); - assertEquals(JobStatus.CANCELLING, strategy.getFailoverRegion(ev11).getState()); - assertEquals(JobStatus.CANCELLING, strategy.getFailoverRegion(ev31).getState()); - - ev32.getCurrentExecutionAttempt().completeCancelling(); - waitUntilFailoverRegionState(strategy.getFailoverRegion(ev31), JobStatus.RUNNING, 1000); - - ev12.getCurrentExecutionAttempt().completeCancelling(); - waitUntilFailoverRegionState(strategy.getFailoverRegion(ev11), JobStatus.RUNNING, 1000); - } - - /** - * Tests that if a task reports the result of its preceding task is failed, - * its preceding task will be considered as failed, and start to failover - * TODO: as the report part is not finished yet, this case is ignored temporarily - * @throws Exception if fail to create dummy job information or fail to schedule for execution. - */ - @Ignore - @Test - public void testSucceedingNoticePreceding() throws Exception { - final JobID jobId = new JobID(); - final String jobName = "Test Job Sample Name"; - - final SimpleSlotProvider slotProvider = new SimpleSlotProvider(jobId, 14); - - JobVertex v1 = new JobVertex("vertex1"); - JobVertex v2 = new JobVertex("vertex2"); - - v1.setParallelism(1); - v2.setParallelism(1); - - v1.setInvokableClass(AbstractInvokable.class); - v2.setInvokableClass(AbstractInvokable.class); - - v2.connectNewDataSetAsInput(v1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - - ExecutionGraph eg = new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(jobId, jobName, v1, v2) - .setRestartStrategy(new InfiniteDelayRestartStrategy(10)) - .setFailoverStrategyFactory(new FailoverPipelinedRegionWithDirectExecutor()) - .setSlotProvider(slotProvider) - .setScheduleMode(ScheduleMode.EAGER) - .build(); - - eg.scheduleForExecution(); - RestartPipelinedRegionStrategy strategy = (RestartPipelinedRegionStrategy)eg.getFailoverStrategy(); - - ExecutionVertex ev11 = eg.getJobVertex(v2.getID()).getTaskVertices()[0]; - ExecutionVertex ev21 = eg.getJobVertex(v2.getID()).getTaskVertices()[0]; - ev21.getCurrentExecutionAttempt().fail(new Exception("Fail with v1")); - - assertEquals(JobStatus.CANCELLING, strategy.getFailoverRegion(ev21).getState()); - assertEquals(JobStatus.CANCELLING, strategy.getFailoverRegion(ev11).getState()); - } - - /** - * Tests that a new failure comes while the failover region is in CANCELLING. - * @throws Exception if fail to create the single region execution graph. - */ - @Test - public void testFailWhileCancelling() throws Exception { - RestartStrategy restartStrategy = new InfiniteDelayRestartStrategy(); - ExecutionGraph eg = createSingleRegionExecutionGraph(restartStrategy); - RestartPipelinedRegionStrategy strategy = (RestartPipelinedRegionStrategy)eg.getFailoverStrategy(); - - Iterator iter = eg.getAllExecutionVertices().iterator(); - ExecutionVertex ev1 = iter.next(); - ev1.getCurrentExecutionAttempt().switchToRunning(); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev1).getState()); - - ev1.getCurrentExecutionAttempt().fail(new Exception("new fail")); - assertEquals(JobStatus.CANCELLING, strategy.getFailoverRegion(ev1).getState()); - - ExecutionVertex ev2 = iter.next(); - ev2.getCurrentExecutionAttempt().fail(new Exception("new fail")); - assertEquals(JobStatus.RUNNING, eg.getState()); - assertEquals(JobStatus.CANCELLING, strategy.getFailoverRegion(ev1).getState()); - } - - /** - * Tests that a new failure comes while the failover region is restarting. - * @throws Exception if fail to create the single region execution graph. - */ - @Test - public void testFailWhileRestarting() throws Exception { - RestartStrategy restartStrategy = new InfiniteDelayRestartStrategy(); - ExecutionGraph eg = createSingleRegionExecutionGraph(restartStrategy); - RestartPipelinedRegionStrategy strategy = (RestartPipelinedRegionStrategy)eg.getFailoverStrategy(); - - Iterator iter = eg.getAllExecutionVertices().iterator(); - ExecutionVertex ev1 = iter.next(); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev1).getState()); - - ev1.getCurrentExecutionAttempt().fail(new Exception("new fail")); - assertEquals(JobStatus.CANCELLING, strategy.getFailoverRegion(ev1).getState()); - - for (ExecutionVertex evs : eg.getAllExecutionVertices()) { - evs.getCurrentExecutionAttempt().completeCancelling(); - } - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev1).getState()); - - ev1.getCurrentExecutionAttempt().fail(new Exception("new fail")); - assertEquals(JobStatus.CANCELLING, strategy.getFailoverRegion(ev1).getState()); - } - - @Test - public void testStatusResettingOnRegionFailover() throws Exception { - final JobID jobId = new JobID(); - final String jobName = "Test Job Sample Name"; - - final SlotProvider slotProvider = new SimpleSlotProvider(jobId, 20); - - JobVertex v1 = new JobVertex("vertex1"); - JobVertex v2 = new JobVertex("vertex2"); - - v1.setParallelism(2); - v2.setParallelism(2); - - v1.setInvokableClass(AbstractInvokable.class); - v2.setInvokableClass(AbstractInvokable.class); - - v2.connectNewDataSetAsInput(v1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - - List ordered = Arrays.asList(v1, v2); - - ExecutionGraph eg = new ExecutionGraph( - new DummyJobInformation( - jobId, - jobName), - TestingUtils.defaultExecutor(), - TestingUtils.defaultExecutor(), - AkkaUtils.getDefaultTimeout(), - new InfiniteDelayRestartStrategy(10), - new FailoverPipelinedRegionWithDirectExecutor(), - slotProvider); - - eg.attachJobGraph(ordered); - eg.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); - - RestartPipelinedRegionStrategy strategy = (RestartPipelinedRegionStrategy)eg.getFailoverStrategy(); - - ExecutionVertex ev11 = eg.getJobVertex(v1.getID()).getTaskVertices()[0]; - ExecutionVertex ev12 = eg.getJobVertex(v1.getID()).getTaskVertices()[1]; - ExecutionVertex ev21 = eg.getJobVertex(v2.getID()).getTaskVertices()[0]; - ExecutionVertex ev22 = eg.getJobVertex(v2.getID()).getTaskVertices()[1]; - - eg.scheduleForExecution(); - - // initial state - assertEquals(ExecutionState.DEPLOYING, ev11.getExecutionState()); - assertEquals(ExecutionState.DEPLOYING, ev12.getExecutionState()); - assertEquals(ExecutionState.CREATED, ev21.getExecutionState()); - assertEquals(ExecutionState.CREATED, ev22.getExecutionState()); - assertFalse(eg.getJobVertex(v1.getID()).getProducedDataSets()[0].areAllPartitionsFinished()); - assertFalse(eg.getJobVertex(v1.getID()).getProducedDataSets()[0].getPartitions()[0].isConsumable()); - assertFalse(eg.getJobVertex(v1.getID()).getProducedDataSets()[0].getPartitions()[1].isConsumable()); - - // partitions all finished - ev11.getCurrentExecutionAttempt().markFinished(); - ev12.getCurrentExecutionAttempt().markFinished(); - assertEquals(ExecutionState.FINISHED, ev11.getExecutionState()); - assertEquals(ExecutionState.FINISHED, ev12.getExecutionState()); - assertEquals(ExecutionState.DEPLOYING, ev21.getExecutionState()); - assertEquals(ExecutionState.DEPLOYING, ev22.getExecutionState()); - assertTrue(eg.getJobVertex(v1.getID()).getProducedDataSets()[0].areAllPartitionsFinished()); - assertTrue(eg.getJobVertex(v1.getID()).getProducedDataSets()[0].getPartitions()[0].isConsumable()); - assertTrue(eg.getJobVertex(v1.getID()).getProducedDataSets()[0].getPartitions()[1].isConsumable()); - - // force the partition producer to restart - strategy.onTaskFailure(ev11.getCurrentExecutionAttempt(), new FlinkException("Fail for testing")); - assertFalse(eg.getJobVertex(v1.getID()).getProducedDataSets()[0].areAllPartitionsFinished()); - assertFalse(eg.getJobVertex(v1.getID()).getProducedDataSets()[0].getPartitions()[0].isConsumable()); - assertFalse(eg.getJobVertex(v1.getID()).getProducedDataSets()[0].getPartitions()[1].isConsumable()); - - // failed partition finishes again - ev11.getCurrentExecutionAttempt().markFinished(); - assertTrue(eg.getJobVertex(v1.getID()).getProducedDataSets()[0].areAllPartitionsFinished()); - assertTrue(eg.getJobVertex(v1.getID()).getProducedDataSets()[0].getPartitions()[0].isConsumable()); - assertTrue(eg.getJobVertex(v1.getID()).getProducedDataSets()[0].getPartitions()[1].isConsumable()); - } - - // -------------------------------------------------------------------------------------------- - - private void verifyCheckpointRestoredAsExpected(ExecutionGraph eg) throws Exception { - // pending checkpoints have already been cancelled. - assertNotNull(eg.getCheckpointCoordinator()); - assertTrue(eg.getCheckpointCoordinator().getPendingCheckpoints().isEmpty()); - - // verify checkpoint has been restored successfully. - assertEquals(1, eg.getCheckpointCoordinator().getCheckpointStore().getNumberOfRetainedCheckpoints()); - assertEquals(checkpointId, eg.getCheckpointCoordinator().getCheckpointStore().getLatestCheckpoint(false).getCheckpointID()); - } - - private ExecutionGraph createSingleRegionExecutionGraph(RestartStrategy restartStrategy) throws Exception { - final JobID jobId = new JobID(); - final String jobName = "Test Job Sample Name"; - - final SimpleSlotProvider slotProvider = new SimpleSlotProvider(jobId, 14); - - JobVertex v1 = new JobVertex("vertex1"); - JobVertex v2 = new JobVertex("vertex2"); - JobVertex v3 = new JobVertex("vertex3"); - - v1.setParallelism(3); - v2.setParallelism(2); - v3.setParallelism(2); - - v1.setInvokableClass(AbstractInvokable.class); - v2.setInvokableClass(AbstractInvokable.class); - v3.setInvokableClass(AbstractInvokable.class); - - v2.connectNewDataSetAsInput(v1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - v3.connectNewDataSetAsInput(v1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - v3.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - List ordered = new ArrayList<>(Arrays.asList(v1, v2, v3)); - - ExecutionGraph eg = new ExecutionGraph( - new DummyJobInformation( - jobId, - jobName), - TestingUtils.defaultExecutor(), - TestingUtils.defaultExecutor(), - AkkaUtils.getDefaultTimeout(), - restartStrategy, - new FailoverPipelinedRegionWithDirectExecutor(), - slotProvider); - try { - eg.attachJobGraph(ordered); - } - catch (JobException e) { - e.printStackTrace(); - fail("Job failed with exception: " + e.getMessage()); - } - - enableCheckpointing(eg); - - eg.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); - eg.scheduleForExecution(); - - attachPendingCheckpoints(eg); - return eg; - } - - // ------------------------------------------------------------------------ - - /** - * A factory to create a RestartPipelinedRegionStrategy that uses a - * direct (synchronous) executor for easier testing. - */ - private static class FailoverPipelinedRegionWithDirectExecutor implements Factory { - - @Override - public FailoverStrategy create(ExecutionGraph executionGraph) { - return new RestartPipelinedRegionStrategy(executionGraph); - } - } - - private static void enableCheckpointing(ExecutionGraph eg) { - ArrayList jobVertices = new ArrayList<>(eg.getAllVertices().values()); - CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( - 1000, - 100, - 0, - 1, - CheckpointRetentionPolicy.RETAIN_ON_CANCELLATION, - true, - false, - 0); - eg.enableCheckpointing( - chkConfig, - jobVertices, - jobVertices, - jobVertices, - Collections.emptyList(), - new StandaloneCheckpointIDCounter(), - new StandaloneCompletedCheckpointStore(1), - new MemoryStateBackend(), - new CheckpointStatsTracker( - 0, - jobVertices, - mock(CheckpointCoordinatorConfiguration.class), - new UnregisteredMetricsGroup())); - } - - /** - * Attach pending checkpoints of chk-42 and chk-43 to the execution graph. - * If {@link #acknowledgeAllCheckpoints(CheckpointCoordinator, Iterator)} called then, - * chk-42 would become the completed checkpoint. - */ - private void attachPendingCheckpoints(ExecutionGraph eg) throws IOException { - final Map pendingCheckpoints = new HashMap<>(); - final Map verticesToConfirm = new HashMap<>(); - eg.getAllExecutionVertices().forEach(e -> { - Execution ee = e.getCurrentExecutionAttempt(); - if (ee != null) { - verticesToConfirm.put(ee.getAttemptId(), e); - } - }); - - CheckpointCoordinator checkpointCoordinator = eg.getCheckpointCoordinator(); - assertNotNull(checkpointCoordinator); - CheckpointStorageCoordinatorView checkpointStorage = checkpointCoordinator.getCheckpointStorage(); - pendingCheckpoints.put(checkpointId, new PendingCheckpoint( - eg.getJobID(), - checkpointId, - 0L, - verticesToConfirm, - CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), - checkpointStorage.initializeLocationForCheckpoint(checkpointId), - eg.getFutureExecutor())); - - long newCheckpointId = checkpointId + 1; - pendingCheckpoints.put(newCheckpointId, new PendingCheckpoint( - eg.getJobID(), - newCheckpointId, - 0L, - verticesToConfirm, - CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), - checkpointStorage.initializeLocationForCheckpoint(newCheckpointId), - eg.getFutureExecutor())); - Whitebox.setInternalState(checkpointCoordinator, "pendingCheckpoints", pendingCheckpoints); - } - - /** - * Let the checkpoint coordinator to receive all acknowledges from given executionVertexes so that to complete the expected checkpoint. - */ - private void acknowledgeAllCheckpoints(CheckpointCoordinator checkpointCoordinator, Iterator executionVertexes) throws IOException, CheckpointException { - while (executionVertexes.hasNext()) { - ExecutionVertex executionVertex = executionVertexes.next(); - for (int index = 0; index < executionVertex.getJobVertex().getParallelism(); index++) { - JobVertexID jobVertexID = executionVertex.getJobvertexId(); - OperatorStateHandle opStateBackend = CheckpointCoordinatorTest.generatePartitionableStateHandle(jobVertexID, index, 2, 8, false); - OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(opStateBackend, null, null, null); - TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); - taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID), operatorSubtaskState); - - AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( - executionVertex.getJobId(), - executionVertex.getJobVertex().getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), - checkpointId, - new CheckpointMetrics(), - taskOperatorSubtaskStates); - - checkpointCoordinator.receiveAcknowledgeMessage(acknowledgeCheckpoint, "Unknown location"); - } - } - } - - private static class CollectTddTaskManagerGateway extends SimpleAckingTaskManagerGateway { - - private final Map attemptIDInitStateMap; - - CollectTddTaskManagerGateway(Map attemptIDInitStateMap) { - this.attemptIDInitStateMap = attemptIDInitStateMap; - } - - @Override - public CompletableFuture submitTask(TaskDeploymentDescriptor tdd, Time timeout) { - attemptIDInitStateMap.put(tdd.getExecutionAttemptId(), tdd.getTaskRestore()); - return super.submitTask(tdd, timeout); - } - } - -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/RestartPipelinedRegionStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/RestartPipelinedRegionStrategyTest.java deleted file mode 100644 index 7b00e4f09f..0000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/RestartPipelinedRegionStrategyTest.java +++ /dev/null @@ -1,388 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.executiongraph; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.JobException; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.blob.VoidBlobWriter; -import org.apache.flink.runtime.executiongraph.failover.FailoverRegion; -import org.apache.flink.runtime.executiongraph.failover.RestartPipelinedRegionStrategy; -import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.jobgraph.DistributionPattern; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.testingUtils.TestingUtils; - -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.CompletableFuture; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.fail; - -public class RestartPipelinedRegionStrategyTest { - - /** - * Creates a JobGraph of the following form: - * - *
-	 *  v1--->v2-->\
-	 *              \
-	 *               v4 --->\
-	 *        ----->/        \
-	 *  v3-->/                v5
-	 *       \               /
-	 *        ------------->/
-	 * 
- */ - @Test - public void testSimpleFailoverRegion() throws Exception { - - final JobID jobId = new JobID(); - final String jobName = "Test Job Sample Name"; - - JobVertex v1 = new JobVertex("vertex1"); - JobVertex v2 = new JobVertex("vertex2"); - JobVertex v3 = new JobVertex("vertex3"); - JobVertex v4 = new JobVertex("vertex4"); - JobVertex v5 = new JobVertex("vertex5"); - - v1.setParallelism(5); - v2.setParallelism(7); - v3.setParallelism(2); - v4.setParallelism(11); - v5.setParallelism(4); - - v1.setInvokableClass(AbstractInvokable.class); - v2.setInvokableClass(AbstractInvokable.class); - v3.setInvokableClass(AbstractInvokable.class); - v4.setInvokableClass(AbstractInvokable.class); - v5.setInvokableClass(AbstractInvokable.class); - - v2.connectNewDataSetAsInput(v1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - v4.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - v4.connectNewDataSetAsInput(v3, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - v5.connectNewDataSetAsInput(v4, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - v5.connectNewDataSetAsInput(v3, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - List ordered = new ArrayList(Arrays.asList(v1, v2, v3, v4, v5)); - - final JobInformation jobInformation = new DummyJobInformation( - jobId, - jobName); - - ExecutionGraph eg = new ExecutionGraph( - jobInformation, - TestingUtils.defaultExecutor(), - TestingUtils.defaultExecutor(), - AkkaUtils.getDefaultTimeout(), - new NoRestartStrategy(), - new RestartPipelinedRegionStrategy.Factory(), - new TestingSlotProvider(ignored -> new CompletableFuture<>()), - ExecutionGraph.class.getClassLoader(), - VoidBlobWriter.getInstance(), - AkkaUtils.getDefaultTimeout()); - try { - eg.attachJobGraph(ordered); - } - catch (JobException e) { - e.printStackTrace(); - fail("Job failed with exception: " + e.getMessage()); - } - - RestartPipelinedRegionStrategy strategy = (RestartPipelinedRegionStrategy)eg.getFailoverStrategy(); - ExecutionJobVertex ejv1 = eg.getJobVertex(v1.getID()); - ExecutionJobVertex ejv2 = eg.getJobVertex(v2.getID()); - ExecutionJobVertex ejv3 = eg.getJobVertex(v3.getID()); - ExecutionJobVertex ejv4 = eg.getJobVertex(v4.getID()); - ExecutionJobVertex ejv5 = eg.getJobVertex(v5.getID()); - FailoverRegion region1 = strategy.getFailoverRegion(ejv1.getTaskVertices()[2]); - FailoverRegion region2 = strategy.getFailoverRegion(ejv2.getTaskVertices()[3]); - FailoverRegion region3 = strategy.getFailoverRegion(ejv3.getTaskVertices()[0]); - FailoverRegion region4 = strategy.getFailoverRegion(ejv4.getTaskVertices()[4]); - FailoverRegion region5 = strategy.getFailoverRegion(ejv5.getTaskVertices()[1]); - - assertEquals(region1, region2); - assertEquals(region3, region2); - assertEquals(region4, region2); - assertEquals(region5, region2); - } - - /** - * Creates a JobGraph of the following form: - * - *
-     *  v2 ------->\
-     *              \
-     *  v1---------> v4 --->|\
-     *                        \
-     *                        v5
-     *                       /
-     *  v3--------------->|/
-     * 
- */ - @Test - public void testMultipleFailoverRegions() throws Exception { - final JobID jobId = new JobID(); - final String jobName = "Test Job Sample Name"; - - JobVertex v1 = new JobVertex("vertex1"); - JobVertex v2 = new JobVertex("vertex2"); - JobVertex v3 = new JobVertex("vertex3"); - JobVertex v4 = new JobVertex("vertex4"); - JobVertex v5 = new JobVertex("vertex5"); - - v1.setParallelism(3); - v2.setParallelism(2); - v3.setParallelism(2); - v4.setParallelism(5); - v5.setParallelism(2); - - v1.setInvokableClass(AbstractInvokable.class); - v2.setInvokableClass(AbstractInvokable.class); - v3.setInvokableClass(AbstractInvokable.class); - v4.setInvokableClass(AbstractInvokable.class); - v5.setInvokableClass(AbstractInvokable.class); - - v4.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - v4.connectNewDataSetAsInput(v1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - v5.connectNewDataSetAsInput(v4, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - v5.connectNewDataSetAsInput(v3, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - - List ordered = new ArrayList(Arrays.asList(v1, v2, v3, v4, v5)); - - final JobInformation jobInformation = new DummyJobInformation( - jobId, - jobName); - - ExecutionGraph eg = new ExecutionGraph( - jobInformation, - TestingUtils.defaultExecutor(), - TestingUtils.defaultExecutor(), - AkkaUtils.getDefaultTimeout(), - new NoRestartStrategy(), - new RestartPipelinedRegionStrategy.Factory(), - new TestingSlotProvider(ignored -> new CompletableFuture<>()), - ExecutionGraph.class.getClassLoader(), - VoidBlobWriter.getInstance(), - AkkaUtils.getDefaultTimeout()); - try { - eg.attachJobGraph(ordered); - } - catch (JobException e) { - e.printStackTrace(); - fail("Job failed with exception: " + e.getMessage()); - } - - // All in one failover region - RestartPipelinedRegionStrategy strategy = (RestartPipelinedRegionStrategy)eg.getFailoverStrategy(); - ExecutionJobVertex ejv1 = eg.getJobVertex(v1.getID()); - ExecutionJobVertex ejv2 = eg.getJobVertex(v2.getID()); - ExecutionJobVertex ejv3 = eg.getJobVertex(v3.getID()); - ExecutionJobVertex ejv4 = eg.getJobVertex(v4.getID()); - ExecutionJobVertex ejv5 = eg.getJobVertex(v5.getID()); - FailoverRegion region1 = strategy.getFailoverRegion(ejv1.getTaskVertices()[1]); - FailoverRegion region2 = strategy.getFailoverRegion(ejv2.getTaskVertices()[0]); - FailoverRegion region4 = strategy.getFailoverRegion(ejv4.getTaskVertices()[3]); - FailoverRegion region31 = strategy.getFailoverRegion(ejv3.getTaskVertices()[0]); - FailoverRegion region32 = strategy.getFailoverRegion(ejv3.getTaskVertices()[1]); - FailoverRegion region51 = strategy.getFailoverRegion(ejv5.getTaskVertices()[0]); - FailoverRegion region52 = strategy.getFailoverRegion(ejv5.getTaskVertices()[1]); - - //There should be 5 failover regions. v1 v2 v4 in one, v3 has two, v5 has two - assertEquals(region1, region2); - assertEquals(region2, region4); - assertFalse(region31.equals(region32)); - assertFalse(region51.equals(region52)); - } - - /** - * Creates a JobGraph of the following form: - * - *
-     *  v1--->v2-->\
-     *              \
-     *               v4 --->|\
-     *        ----->/        \
-     *  v3-->/                v5
-     *       \               /
-     *        ------------->/
-     * 
- */ - @Test - public void testSingleRegionWithMixedInput() throws Exception { - final JobID jobId = new JobID(); - final String jobName = "Test Job Sample Name"; - - JobVertex v1 = new JobVertex("vertex1"); - JobVertex v2 = new JobVertex("vertex2"); - JobVertex v3 = new JobVertex("vertex3"); - JobVertex v4 = new JobVertex("vertex4"); - JobVertex v5 = new JobVertex("vertex5"); - - v1.setParallelism(3); - v2.setParallelism(2); - v3.setParallelism(2); - v4.setParallelism(5); - v5.setParallelism(2); - - v1.setInvokableClass(AbstractInvokable.class); - v2.setInvokableClass(AbstractInvokable.class); - v3.setInvokableClass(AbstractInvokable.class); - v4.setInvokableClass(AbstractInvokable.class); - v5.setInvokableClass(AbstractInvokable.class); - - v2.connectNewDataSetAsInput(v1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - v4.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - v4.connectNewDataSetAsInput(v3, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - v5.connectNewDataSetAsInput(v3, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - v5.connectNewDataSetAsInput(v4, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - - List ordered = new ArrayList(Arrays.asList(v1, v2, v3, v4, v5)); - - final JobInformation jobInformation = new DummyJobInformation( - jobId, - jobName); - - ExecutionGraph eg = new ExecutionGraph( - jobInformation, - TestingUtils.defaultExecutor(), - TestingUtils.defaultExecutor(), - AkkaUtils.getDefaultTimeout(), - new NoRestartStrategy(), - new RestartPipelinedRegionStrategy.Factory(), - new TestingSlotProvider(ignored -> new CompletableFuture<>()), - ExecutionGraph.class.getClassLoader(), - VoidBlobWriter.getInstance(), - AkkaUtils.getDefaultTimeout()); - try { - eg.attachJobGraph(ordered); - } - catch (JobException e) { - e.printStackTrace(); - fail("Job failed with exception: " + e.getMessage()); - } - - // All in one failover region - RestartPipelinedRegionStrategy strategy = (RestartPipelinedRegionStrategy)eg.getFailoverStrategy(); - ExecutionJobVertex ejv1 = eg.getJobVertex(v1.getID()); - ExecutionJobVertex ejv2 = eg.getJobVertex(v2.getID()); - ExecutionJobVertex ejv3 = eg.getJobVertex(v3.getID()); - ExecutionJobVertex ejv4 = eg.getJobVertex(v4.getID()); - ExecutionJobVertex ejv5 = eg.getJobVertex(v5.getID()); - FailoverRegion region1 = strategy.getFailoverRegion(ejv1.getTaskVertices()[1]); - FailoverRegion region2 = strategy.getFailoverRegion(ejv2.getTaskVertices()[0]); - FailoverRegion region4 = strategy.getFailoverRegion(ejv4.getTaskVertices()[3]); - FailoverRegion region3 = strategy.getFailoverRegion(ejv3.getTaskVertices()[0]); - FailoverRegion region5 = strategy.getFailoverRegion(ejv5.getTaskVertices()[1]); - - assertEquals(region1, region2); - assertEquals(region2, region4); - assertEquals(region3, region2); - assertEquals(region1, region5); - } - - /** - * Creates a JobGraph of the following form: - * - *
-     *  v1-->v2-->|\
-     *              \
-     *               v4
-     *             /
-     *  v3------>/
-     * 
- */ - @Test - public void testMultiRegionNotAllToAll() throws Exception { - final JobID jobId = new JobID(); - final String jobName = "Test Job Sample Name"; - - JobVertex v1 = new JobVertex("vertex1"); - JobVertex v2 = new JobVertex("vertex2"); - JobVertex v3 = new JobVertex("vertex3"); - JobVertex v4 = new JobVertex("vertex4"); - JobVertex v5 = new JobVertex("vertex5"); - - v1.setParallelism(2); - v2.setParallelism(2); - v3.setParallelism(5); - v4.setParallelism(5); - - v1.setInvokableClass(AbstractInvokable.class); - v2.setInvokableClass(AbstractInvokable.class); - v3.setInvokableClass(AbstractInvokable.class); - v4.setInvokableClass(AbstractInvokable.class); - - v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED); - v4.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - v4.connectNewDataSetAsInput(v3, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - - List ordered = new ArrayList(Arrays.asList(v1, v2, v3, v4)); - - final JobInformation jobInformation = new DummyJobInformation( - jobId, - jobName); - - ExecutionGraph eg = new ExecutionGraph( - jobInformation, - TestingUtils.defaultExecutor(), - TestingUtils.defaultExecutor(), - AkkaUtils.getDefaultTimeout(), - new NoRestartStrategy(), - new RestartPipelinedRegionStrategy.Factory(), - new TestingSlotProvider(ignored -> new CompletableFuture<>()), - ExecutionGraph.class.getClassLoader(), - VoidBlobWriter.getInstance(), - AkkaUtils.getDefaultTimeout()); - try { - eg.attachJobGraph(ordered); - } - catch (JobException e) { - e.printStackTrace(); - fail("Job failed with exception: " + e.getMessage()); - } - - // All in one failover region - RestartPipelinedRegionStrategy strategy = (RestartPipelinedRegionStrategy)eg.getFailoverStrategy(); - ExecutionJobVertex ejv1 = eg.getJobVertex(v1.getID()); - ExecutionJobVertex ejv2 = eg.getJobVertex(v2.getID()); - ExecutionJobVertex ejv3 = eg.getJobVertex(v3.getID()); - ExecutionJobVertex ejv4 = eg.getJobVertex(v4.getID()); - FailoverRegion region11 = strategy.getFailoverRegion(ejv1.getTaskVertices()[0]); - FailoverRegion region12 = strategy.getFailoverRegion(ejv1.getTaskVertices()[1]); - FailoverRegion region21 = strategy.getFailoverRegion(ejv2.getTaskVertices()[0]); - FailoverRegion region22 = strategy.getFailoverRegion(ejv2.getTaskVertices()[1]); - FailoverRegion region3 = strategy.getFailoverRegion(ejv3.getTaskVertices()[0]); - FailoverRegion region4 = strategy.getFailoverRegion(ejv4.getTaskVertices()[3]); - - //There should be 3 failover regions. v1 v2 in two, v3 and v4 in one - assertEquals(region11, region21); - assertEquals(region12, region22); - assertFalse(region11.equals(region12)); - assertFalse(region3.equals(region4)); - } - -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/PipelinedFailoverRegionBuildingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/PipelinedFailoverRegionBuildingTest.java deleted file mode 100644 index f17dddcfb0..0000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/PipelinedFailoverRegionBuildingTest.java +++ /dev/null @@ -1,651 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.executiongraph.failover; - -import org.apache.flink.api.common.time.Time; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; -import org.apache.flink.runtime.JobException; -import org.apache.flink.runtime.blob.VoidBlobWriter; -import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; -import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.runtime.executiongraph.ExecutionGraph; -import org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder; -import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; -import org.apache.flink.runtime.io.network.partition.NoOpPartitionTracker; -import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.jobgraph.DistributionPattern; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; -import org.apache.flink.runtime.shuffle.NettyShuffleMaster; -import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.runtime.testtasks.NoOpInvokable; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; - -import java.util.Iterator; - -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; - -/** - * Tests that make sure that the building of pipelined connected failover regions works - * correctly. - */ -public class PipelinedFailoverRegionBuildingTest extends TestLogger { - - /** - * Tests that validates that a graph with single unconnected vertices works correctly. - * - *
-	 *     (v1)
-	 *     
-	 *     (v2)
-	 *     
-	 *     (v3)
-	 *     
-	 *     ...
-	 * 
- */ - @Test - public void testIndividualVertices() throws Exception { - final JobVertex source1 = new JobVertex("source1"); - source1.setInvokableClass(NoOpInvokable.class); - source1.setParallelism(2); - - final JobVertex source2 = new JobVertex("source2"); - source2.setInvokableClass(NoOpInvokable.class); - source2.setParallelism(2); - - final JobGraph jobGraph = new JobGraph("test job", source1, source2); - final ExecutionGraph eg = createExecutionGraph(jobGraph); - - RestartPipelinedRegionStrategy failoverStrategy = (RestartPipelinedRegionStrategy) eg.getFailoverStrategy(); - FailoverRegion sourceRegion11 = failoverStrategy.getFailoverRegion(eg.getJobVertex(source1.getID()).getTaskVertices()[0]); - FailoverRegion sourceRegion12 = failoverStrategy.getFailoverRegion(eg.getJobVertex(source1.getID()).getTaskVertices()[1]); - FailoverRegion targetRegion21 = failoverStrategy.getFailoverRegion(eg.getJobVertex(source2.getID()).getTaskVertices()[0]); - FailoverRegion targetRegion22 = failoverStrategy.getFailoverRegion(eg.getJobVertex(source2.getID()).getTaskVertices()[1]); - - assertTrue(sourceRegion11 != sourceRegion12); - assertTrue(sourceRegion12 != targetRegion21); - assertTrue(targetRegion21 != targetRegion22); - } - - /** - * Tests that validates that embarrassingly parallel chains of vertices work correctly. - * - *
-	 *     (a1) --> (b1)
-	 *
-	 *     (a2) --> (b2)
-	 *
-	 *     (a3) --> (b3)
-	 *
-	 *     ...
-	 * 
- */ - @Test - public void testEmbarrassinglyParallelCase() throws Exception { - int parallelism = 10000; - final JobVertex vertex1 = new JobVertex("vertex1"); - vertex1.setInvokableClass(NoOpInvokable.class); - vertex1.setParallelism(parallelism); - - final JobVertex vertex2 = new JobVertex("vertex2"); - vertex2.setInvokableClass(NoOpInvokable.class); - vertex2.setParallelism(parallelism); - - final JobVertex vertex3 = new JobVertex("vertex3"); - vertex3.setInvokableClass(NoOpInvokable.class); - vertex3.setParallelism(parallelism); - - vertex2.connectNewDataSetAsInput(vertex1, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED); - vertex3.connectNewDataSetAsInput(vertex2, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED); - - final JobGraph jobGraph = new JobGraph("test job", vertex1, vertex2, vertex3); - final ExecutionGraph eg = createExecutionGraph(jobGraph); - - RestartPipelinedRegionStrategy failoverStrategy = (RestartPipelinedRegionStrategy) eg.getFailoverStrategy(); - FailoverRegion preRegion1 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex1.getID()).getTaskVertices()[0]); - FailoverRegion preRegion2 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex2.getID()).getTaskVertices()[0]); - FailoverRegion preRegion3 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex3.getID()).getTaskVertices()[0]); - - assertTrue(preRegion1 == preRegion2); - assertTrue(preRegion2 == preRegion3); - - for (int i = 1; i < parallelism; ++i) { - FailoverRegion region1 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex1.getID()).getTaskVertices()[i]); - FailoverRegion region2 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex2.getID()).getTaskVertices()[i]); - FailoverRegion region3 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex3.getID()).getTaskVertices()[i]); - - assertTrue(region1 == region2); - assertTrue(region2 == region3); - - assertTrue(preRegion1 != region1); - } - } - - /** - * Tests that validates that a single pipelined component via a sequence of all-to-all - * connections works correctly. - * - *
-	 *     (a1) -+-> (b1) -+-> (c1) 
-	 *           X         X
-	 *     (a2) -+-> (b2) -+-> (c2)
-	 *           X         X
-	 *     (a3) -+-> (b3) -+-> (c3)
-	 *
-	 *     ...
-	 * 
- */ - @Test - public void testOneComponentViaTwoExchanges() throws Exception { - final JobVertex vertex1 = new JobVertex("vertex1"); - vertex1.setInvokableClass(NoOpInvokable.class); - vertex1.setParallelism(3); - - final JobVertex vertex2 = new JobVertex("vertex2"); - vertex2.setInvokableClass(NoOpInvokable.class); - vertex2.setParallelism(5); - - final JobVertex vertex3 = new JobVertex("vertex3"); - vertex3.setInvokableClass(NoOpInvokable.class); - vertex3.setParallelism(2); - - vertex2.connectNewDataSetAsInput(vertex1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - vertex3.connectNewDataSetAsInput(vertex2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - final JobGraph jobGraph = new JobGraph("test job", vertex1, vertex2, vertex3); - final ExecutionGraph eg = createExecutionGraph(jobGraph); - - RestartPipelinedRegionStrategy failoverStrategy = (RestartPipelinedRegionStrategy) eg.getFailoverStrategy(); - FailoverRegion region1 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex1.getID()).getTaskVertices()[1]); - FailoverRegion region2 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex2.getID()).getTaskVertices()[4]); - FailoverRegion region3 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex3.getID()).getTaskVertices()[0]); - - assertTrue(region1 == region2); - assertTrue(region2 == region3); - } - - /** - * Tests that validates that a single pipelined component via a cascade of joins - * works correctly. - * - *

Non-parallelized view: - *

-	 *     (1)--+
-	 *          +--(5)-+
-	 *     (2)--+      |
-	 *                 +--(7)
-	 *     (3)--+      |
-	 *          +--(6)-+
-	 *     (4)--+
-	 *     ...
-	 * 
- */ - @Test - public void testOneComponentViaCascadeOfJoins() throws Exception { - final JobVertex vertex1 = new JobVertex("vertex1"); - vertex1.setInvokableClass(NoOpInvokable.class); - vertex1.setParallelism(8); - - final JobVertex vertex2 = new JobVertex("vertex2"); - vertex2.setInvokableClass(NoOpInvokable.class); - vertex2.setParallelism(8); - - final JobVertex vertex3 = new JobVertex("vertex3"); - vertex3.setInvokableClass(NoOpInvokable.class); - vertex3.setParallelism(8); - - final JobVertex vertex4 = new JobVertex("vertex4"); - vertex4.setInvokableClass(NoOpInvokable.class); - vertex4.setParallelism(8); - - final JobVertex vertex5 = new JobVertex("vertex5"); - vertex5.setInvokableClass(NoOpInvokable.class); - vertex5.setParallelism(4); - - final JobVertex vertex6 = new JobVertex("vertex6"); - vertex6.setInvokableClass(NoOpInvokable.class); - vertex6.setParallelism(4); - - final JobVertex vertex7 = new JobVertex("vertex7"); - vertex7.setInvokableClass(NoOpInvokable.class); - vertex7.setParallelism(2); - - vertex5.connectNewDataSetAsInput(vertex1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - vertex5.connectNewDataSetAsInput(vertex2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - vertex6.connectNewDataSetAsInput(vertex3, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - vertex6.connectNewDataSetAsInput(vertex4, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - vertex7.connectNewDataSetAsInput(vertex5, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - vertex7.connectNewDataSetAsInput(vertex6, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - final JobGraph jobGraph = new JobGraph("test job", vertex1, vertex2, vertex3, vertex4, vertex5, vertex6, vertex7); - final ExecutionGraph eg = createExecutionGraph(jobGraph); - - RestartPipelinedRegionStrategy failoverStrategy = (RestartPipelinedRegionStrategy) eg.getFailoverStrategy(); - - Iterator evs = eg.getAllExecutionVertices().iterator(); - - FailoverRegion preRegion = failoverStrategy.getFailoverRegion(evs.next()); - - while (evs.hasNext()) { - FailoverRegion region = failoverStrategy.getFailoverRegion(evs.next()); - assertTrue(preRegion == region); - } - } - - /** - * Tests that validates that a single pipelined component instance from one source - * works correctly. - * - *

Non-parallelized view: - *

-	 *                 +--(1)
-	 *          +--(5)-+
-	 *          |      +--(2)
-	 *     (7)--+
-	 *          |      +--(3)
-	 *          +--(6)-+
-	 *                 +--(4)
-	 *     ...
-	 * 
- */ - @Test - public void testOneComponentInstanceFromOneSource() throws Exception { - final JobVertex vertex1 = new JobVertex("vertex1"); - vertex1.setInvokableClass(NoOpInvokable.class); - vertex1.setParallelism(8); - - final JobVertex vertex2 = new JobVertex("vertex2"); - vertex2.setInvokableClass(NoOpInvokable.class); - vertex2.setParallelism(8); - - final JobVertex vertex3 = new JobVertex("vertex3"); - vertex3.setInvokableClass(NoOpInvokable.class); - vertex3.setParallelism(8); - - final JobVertex vertex4 = new JobVertex("vertex4"); - vertex4.setInvokableClass(NoOpInvokable.class); - vertex4.setParallelism(8); - - final JobVertex vertex5 = new JobVertex("vertex5"); - vertex5.setInvokableClass(NoOpInvokable.class); - vertex5.setParallelism(4); - - final JobVertex vertex6 = new JobVertex("vertex6"); - vertex6.setInvokableClass(NoOpInvokable.class); - vertex6.setParallelism(4); - - final JobVertex vertex7 = new JobVertex("vertex7"); - vertex7.setInvokableClass(NoOpInvokable.class); - vertex7.setParallelism(2); - - vertex1.connectNewDataSetAsInput(vertex5, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - vertex2.connectNewDataSetAsInput(vertex5, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - vertex3.connectNewDataSetAsInput(vertex6, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - vertex4.connectNewDataSetAsInput(vertex6, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - vertex5.connectNewDataSetAsInput(vertex7, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - vertex6.connectNewDataSetAsInput(vertex7, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - final JobGraph jobGraph = new JobGraph("test job", vertex7, vertex5, vertex6, vertex1, vertex2, vertex3, vertex4); - final ExecutionGraph eg = createExecutionGraph(jobGraph); - - RestartPipelinedRegionStrategy failoverStrategy = (RestartPipelinedRegionStrategy) eg.getFailoverStrategy(); - - Iterator evs = eg.getAllExecutionVertices().iterator(); - - FailoverRegion preRegion = failoverStrategy.getFailoverRegion(evs.next()); - - while (evs.hasNext()) { - FailoverRegion region = failoverStrategy.getFailoverRegion(evs.next()); - assertTrue(preRegion == region); - } - } - - /** - *
-	 *     (a1) -+-> (b1) -+-> (c1) 
-	 *           X
-	 *     (a2) -+-> (b2) -+-> (c2)
-	 *           X
-	 *     (a3) -+-> (b3) -+-> (c3)
-	 *
-	 *           ^         ^
-	 *           |         |
-	 *     (pipelined) (blocking)
-	 *
-	 * 
- */ - @Test - public void testTwoComponentsViaBlockingExchange() throws Exception { - final JobVertex vertex1 = new JobVertex("vertex1"); - vertex1.setInvokableClass(NoOpInvokable.class); - vertex1.setParallelism(3); - - final JobVertex vertex2 = new JobVertex("vertex2"); - vertex2.setInvokableClass(NoOpInvokable.class); - vertex2.setParallelism(2); - - final JobVertex vertex3 = new JobVertex("vertex3"); - vertex3.setInvokableClass(NoOpInvokable.class); - vertex3.setParallelism(2); - - vertex2.connectNewDataSetAsInput(vertex1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - vertex3.connectNewDataSetAsInput(vertex2, DistributionPattern.POINTWISE, ResultPartitionType.BLOCKING); - - final JobGraph jobGraph = new JobGraph("test job", vertex1, vertex2, vertex3); - final ExecutionGraph eg = createExecutionGraph(jobGraph); - - RestartPipelinedRegionStrategy failoverStrategy = (RestartPipelinedRegionStrategy) eg.getFailoverStrategy(); - FailoverRegion region1 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex1.getID()).getTaskVertices()[1]); - FailoverRegion region2 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex2.getID()).getTaskVertices()[0]); - FailoverRegion region31 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex3.getID()).getTaskVertices()[0]); - FailoverRegion region32 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex3.getID()).getTaskVertices()[1]); - - assertTrue(region1 == region2); - assertTrue(region2 != region31); - assertTrue(region32 != region31); - } - - /** - *
-	 *     (a1) -+-> (b1) -+-> (c1) 
-	 *           X         X
-	 *     (a2) -+-> (b2) -+-> (c2)
-	 *           X         X
-	 *     (a3) -+-> (b3) -+-> (c3)
-	 *
-	 *           ^         ^
-	 *           |         |
-	 *     (pipelined) (blocking)
-	 * 
- */ - @Test - public void testTwoComponentsViaBlockingExchange2() throws Exception { - final JobVertex vertex1 = new JobVertex("vertex1"); - vertex1.setInvokableClass(NoOpInvokable.class); - vertex1.setParallelism(3); - - final JobVertex vertex2 = new JobVertex("vertex2"); - vertex2.setInvokableClass(NoOpInvokable.class); - vertex2.setParallelism(2); - - final JobVertex vertex3 = new JobVertex("vertex3"); - vertex3.setInvokableClass(NoOpInvokable.class); - vertex3.setParallelism(2); - - vertex2.connectNewDataSetAsInput(vertex1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - vertex3.connectNewDataSetAsInput(vertex2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - - final JobGraph jobGraph = new JobGraph("test job", vertex1, vertex2, vertex3); - final ExecutionGraph eg = createExecutionGraph(jobGraph); - - RestartPipelinedRegionStrategy failoverStrategy = (RestartPipelinedRegionStrategy) eg.getFailoverStrategy(); - FailoverRegion region1 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex1.getID()).getTaskVertices()[1]); - FailoverRegion region2 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex2.getID()).getTaskVertices()[0]); - FailoverRegion region31 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex3.getID()).getTaskVertices()[0]); - FailoverRegion region32 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex3.getID()).getTaskVertices()[1]); - - assertTrue(region1 == region2); - assertTrue(region2 != region31); - assertTrue(region32 != region31); - } - - /** - * Cascades of joins with partially blocking, partially pipelined exchanges: - *
-	 *     (1)--+
-	 *          +--(5)-+
-	 *     (2)--+      |
-	 *              (block)
-	 *                 |
-	 *                 +--(7)
-	 *                 |
-	 *              (block)
-	 *     (3)--+      |
-	 *          +--(6)-+
-	 *     (4)--+
-	 *     ...
-	 * 
- * - * Component 1: 1, 2, 5; component 2: 3,4,6; component 3: 7 - */ - @Test - public void testMultipleComponentsViaCascadeOfJoins() throws Exception { - final JobVertex vertex1 = new JobVertex("vertex1"); - vertex1.setInvokableClass(NoOpInvokable.class); - vertex1.setParallelism(8); - - final JobVertex vertex2 = new JobVertex("vertex2"); - vertex2.setInvokableClass(NoOpInvokable.class); - vertex2.setParallelism(8); - - final JobVertex vertex3 = new JobVertex("vertex3"); - vertex3.setInvokableClass(NoOpInvokable.class); - vertex3.setParallelism(8); - - final JobVertex vertex4 = new JobVertex("vertex4"); - vertex4.setInvokableClass(NoOpInvokable.class); - vertex4.setParallelism(8); - - final JobVertex vertex5 = new JobVertex("vertex5"); - vertex5.setInvokableClass(NoOpInvokable.class); - vertex5.setParallelism(4); - - final JobVertex vertex6 = new JobVertex("vertex6"); - vertex6.setInvokableClass(NoOpInvokable.class); - vertex6.setParallelism(4); - - final JobVertex vertex7 = new JobVertex("vertex7"); - vertex7.setInvokableClass(NoOpInvokable.class); - vertex7.setParallelism(2); - - vertex5.connectNewDataSetAsInput(vertex1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - vertex5.connectNewDataSetAsInput(vertex2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - vertex6.connectNewDataSetAsInput(vertex3, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - vertex6.connectNewDataSetAsInput(vertex4, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - vertex7.connectNewDataSetAsInput(vertex5, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - vertex7.connectNewDataSetAsInput(vertex6, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - - final JobGraph jobGraph = new JobGraph("test job", vertex1, vertex2, vertex3, vertex4, vertex5, vertex6, vertex7); - final ExecutionGraph eg = createExecutionGraph(jobGraph); - - RestartPipelinedRegionStrategy failoverStrategy = (RestartPipelinedRegionStrategy) eg.getFailoverStrategy(); - - FailoverRegion region1 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex1.getID()).getTaskVertices()[0]); - FailoverRegion region2 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex2.getID()).getTaskVertices()[5]); - FailoverRegion region5 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex5.getID()).getTaskVertices()[2]); - - assertTrue(region1 == region2); - assertTrue(region1 == region5); - - FailoverRegion region3 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex3.getID()).getTaskVertices()[0]); - FailoverRegion region4 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex4.getID()).getTaskVertices()[5]); - FailoverRegion region6 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex6.getID()).getTaskVertices()[2]); - - assertTrue(region3 == region4); - assertTrue(region3 == region6); - - FailoverRegion region71 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex7.getID()).getTaskVertices()[0]); - FailoverRegion region72 = failoverStrategy.getFailoverRegion(eg.getJobVertex(vertex7.getID()).getTaskVertices()[1]); - - assertTrue(region71 != region72); - assertTrue(region1 != region71); - assertTrue(region1 != region72); - assertTrue(region3 != region71); - assertTrue(region3 != region72); - } - - @Test - public void testDiamondWithMixedPipelinedAndBlockingExchanges() throws Exception { - final JobVertex vertex1 = new JobVertex("vertex1"); - vertex1.setInvokableClass(NoOpInvokable.class); - vertex1.setParallelism(8); - - final JobVertex vertex2 = new JobVertex("vertex2"); - vertex2.setInvokableClass(NoOpInvokable.class); - vertex2.setParallelism(8); - - final JobVertex vertex3 = new JobVertex("vertex3"); - vertex3.setInvokableClass(NoOpInvokable.class); - vertex3.setParallelism(8); - - final JobVertex vertex4 = new JobVertex("vertex4"); - vertex4.setInvokableClass(NoOpInvokable.class); - vertex4.setParallelism(8); - - vertex2.connectNewDataSetAsInput(vertex1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - vertex3.connectNewDataSetAsInput(vertex1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - vertex4.connectNewDataSetAsInput(vertex2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - vertex4.connectNewDataSetAsInput(vertex3, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - - final JobGraph jobGraph = new JobGraph("test job", vertex1, vertex2, vertex3, vertex4); - final ExecutionGraph eg = createExecutionGraph(jobGraph); - - RestartPipelinedRegionStrategy failoverStrategy = (RestartPipelinedRegionStrategy) eg.getFailoverStrategy(); - - Iterator evs = eg.getAllExecutionVertices().iterator(); - - FailoverRegion preRegion = failoverStrategy.getFailoverRegion(evs.next()); - - while (evs.hasNext()) { - FailoverRegion region = failoverStrategy.getFailoverRegion(evs.next()); - assertTrue(preRegion == region); - } - } - - /** - * This test checks that are strictly co-located vertices are in the same failover region, - * even through they are connected via a blocking pattern. - * This is currently an assumption / limitation of the scheduler. - */ - @Test - public void testBlockingAllToAllTopologyWithCoLocation() throws Exception { - final JobVertex source = new JobVertex("source"); - source.setInvokableClass(NoOpInvokable.class); - source.setParallelism(10); - - final JobVertex target = new JobVertex("target"); - target.setInvokableClass(NoOpInvokable.class); - target.setParallelism(13); - - target.connectNewDataSetAsInput(source, DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - - final SlotSharingGroup sharingGroup = new SlotSharingGroup(); - source.setSlotSharingGroup(sharingGroup); - target.setSlotSharingGroup(sharingGroup); - - source.setStrictlyCoLocatedWith(target); - - final JobGraph jobGraph = new JobGraph("test job", source, target); - final ExecutionGraph eg = createExecutionGraph(jobGraph); - - RestartPipelinedRegionStrategy failoverStrategy = (RestartPipelinedRegionStrategy) eg.getFailoverStrategy(); - FailoverRegion region1 = failoverStrategy.getFailoverRegion(eg.getJobVertex(source.getID()).getTaskVertices()[0]); - FailoverRegion region2 = failoverStrategy.getFailoverRegion(eg.getJobVertex(target.getID()).getTaskVertices()[0]); - - // we use 'assertTrue' here rather than 'assertEquals' because we want to test - // for referential equality, to be on the safe side - assertTrue(region1 == region2); - } - - /** - * This test checks that are strictly co-located vertices are in the same failover region, - * even through they are connected via a blocking pattern. - * This is currently an assumption / limitation of the scheduler. - */ - @Test - public void testPipelinedOneToOneTopologyWithCoLocation() throws Exception { - final JobVertex source = new JobVertex("source"); - source.setInvokableClass(NoOpInvokable.class); - source.setParallelism(10); - - final JobVertex target = new JobVertex("target"); - target.setInvokableClass(NoOpInvokable.class); - target.setParallelism(10); - - target.connectNewDataSetAsInput(source, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED); - - final SlotSharingGroup sharingGroup = new SlotSharingGroup(); - source.setSlotSharingGroup(sharingGroup); - target.setSlotSharingGroup(sharingGroup); - - source.setStrictlyCoLocatedWith(target); - - final JobGraph jobGraph = new JobGraph("test job", source, target); - final ExecutionGraph eg = createExecutionGraph(jobGraph); - - RestartPipelinedRegionStrategy failoverStrategy = (RestartPipelinedRegionStrategy) eg.getFailoverStrategy(); - FailoverRegion sourceRegion1 = failoverStrategy.getFailoverRegion(eg.getJobVertex(source.getID()).getTaskVertices()[0]); - FailoverRegion sourceRegion2 = failoverStrategy.getFailoverRegion(eg.getJobVertex(source.getID()).getTaskVertices()[1]); - FailoverRegion targetRegion1 = failoverStrategy.getFailoverRegion(eg.getJobVertex(target.getID()).getTaskVertices()[0]); - FailoverRegion targetRegion2 = failoverStrategy.getFailoverRegion(eg.getJobVertex(target.getID()).getTaskVertices()[1]); - - // we use 'assertTrue' here rather than 'assertEquals' because we want to test - // for referential equality, to be on the safe side - assertTrue(sourceRegion1 == sourceRegion2); - assertTrue(sourceRegion2 == targetRegion1); - assertTrue(targetRegion1 == targetRegion2); - } - - // ------------------------------------------------------------------------ - // utilities - // ------------------------------------------------------------------------ - - private ExecutionGraph createExecutionGraph(JobGraph jobGraph) throws JobException, JobExecutionException { - // configure the pipelined failover strategy - final Configuration jobManagerConfig = new Configuration(); - jobManagerConfig.setString( - JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, - FailoverStrategyLoader.LEGACY_PIPELINED_REGION_RESTART_STRATEGY_NAME); - - final Time timeout = Time.seconds(10L); - return ExecutionGraphBuilder.buildGraph( - null, - jobGraph, - jobManagerConfig, - TestingUtils.defaultExecutor(), - TestingUtils.defaultExecutor(), - mock(SlotProvider.class), - PipelinedFailoverRegionBuildingTest.class.getClassLoader(), - new StandaloneCheckpointRecoveryFactory(), - timeout, - new NoRestartStrategy(), - new UnregisteredMetricsGroup(), - VoidBlobWriter.getInstance(), - timeout, - log, - NettyShuffleMaster.INSTANCE, - NoOpPartitionTracker.INSTANCE); - } -} -- Gitee From 4064b5b67d6d220e1d5518bca96688f51cbbb891 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 4 Oct 2019 13:56:44 +0200 Subject: [PATCH 153/268] [FLINK-14315] Make heartbeat manager fields non-nullable This commit introduces the NoOpHeartbeatManager which can be used to initialize an unset heartbeat manager field. This allows to make the heartbeat manager fields non-nullable which in turn avoid NPE. Moreover, this commit makes the heartbeat manager fields of the TaskExecutor final. This closes #9837. --- .../heartbeat/NoOpHeartbeatManager.java | 58 +++++++++++++++++++ .../flink/runtime/jobmaster/JobMaster.java | 14 ++--- .../resourcemanager/ResourceManager.java | 11 ++-- .../runtime/taskexecutor/TaskExecutor.java | 58 +++++++------------ 4 files changed, 89 insertions(+), 52 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/NoOpHeartbeatManager.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/NoOpHeartbeatManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/NoOpHeartbeatManager.java new file mode 100644 index 0000000000..965a50b3f3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/NoOpHeartbeatManager.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.runtime.heartbeat; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; + +/** + * {@link HeartbeatManager} implementation which does nothing. + * + * @param ignored + * @param ignored + */ +public class NoOpHeartbeatManager implements HeartbeatManager { + private static final NoOpHeartbeatManager INSTANCE = new NoOpHeartbeatManager<>(); + + private NoOpHeartbeatManager() {} + + @Override + public void monitorTarget(ResourceID resourceID, HeartbeatTarget heartbeatTarget) {} + + @Override + public void unmonitorTarget(ResourceID resourceID) {} + + @Override + public void stop() {} + + @Override + public long getLastHeartbeatFrom(ResourceID resourceId) { + return 0; + } + + @Override + public void receiveHeartbeat(ResourceID heartbeatOrigin, I heartbeatPayload) {} + + @Override + public void requestHeartbeat(ResourceID requestOrigin, I heartbeatPayload) {} + + @SuppressWarnings("unchecked") + public static NoOpHeartbeatManager getInstance() { + return (NoOpHeartbeatManager) INSTANCE; + } +} 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 6b6b85cf8d..665c4aa479 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 @@ -38,6 +38,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatListener; import org.apache.flink.runtime.heartbeat.HeartbeatManager; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.heartbeat.HeartbeatTarget; +import org.apache.flink.runtime.heartbeat.NoOpHeartbeatManager; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.io.network.partition.PartitionTracker; import org.apache.flink.runtime.io.network.partition.PartitionTrackerFactory; @@ -269,6 +270,8 @@ public class JobMaster extends FencedRpcEndpoint implements JobMast this.establishedResourceManagerConnection = null; this.accumulators = new HashMap<>(); + this.taskManagerHeartbeatManager = NoOpHeartbeatManager.getInstance(); + this.resourceManagerHeartbeatManager = NoOpHeartbeatManager.getInstance(); } private SchedulerNG createScheduler(final JobManagerJobMetricGroup jobManagerJobMetricGroup) throws Exception { @@ -785,15 +788,8 @@ public class JobMaster extends FencedRpcEndpoint implements JobMast } private void stopHeartbeatServices() { - if (taskManagerHeartbeatManager != null) { - taskManagerHeartbeatManager.stop(); - taskManagerHeartbeatManager = null; - } - - if (resourceManagerHeartbeatManager != null) { - resourceManagerHeartbeatManager.stop(); - resourceManagerHeartbeatManager = null; - } + taskManagerHeartbeatManager.stop(); + resourceManagerHeartbeatManager.stop(); } private void startHeartbeatServices() { 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 543a5c3137..8698e842ac 100755 --- 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 @@ -38,6 +38,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatListener; import org.apache.flink.runtime.heartbeat.HeartbeatManager; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.heartbeat.HeartbeatTarget; +import org.apache.flink.runtime.heartbeat.NoOpHeartbeatManager; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.InstanceID; @@ -178,6 +179,9 @@ public abstract class ResourceManager this.jmResourceIdRegistrations = new HashMap<>(4); this.taskExecutors = new HashMap<>(8); this.taskExecutorGatewayFutures = new HashMap<>(8); + + this.jobManagerHeartbeatManager = NoOpHeartbeatManager.getInstance(); + this.taskManagerHeartbeatManager = NoOpHeartbeatManager.getInstance(); } @@ -972,15 +976,8 @@ public abstract class ResourceManager } private void stopHeartbeatServices() { - if (taskManagerHeartbeatManager != null) { taskManagerHeartbeatManager.stop(); - taskManagerHeartbeatManager = null; - } - - if (jobManagerHeartbeatManager != null) { jobManagerHeartbeatManager.stop(); - jobManagerHeartbeatManager = null; - } } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index b1238ec8e8..f0db4cd82b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -157,8 +157,6 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { /** The task manager configuration. */ private final TaskManagerConfiguration taskManagerConfiguration; - private final HeartbeatServices heartbeatServices; - /** The fatal error handler to use in case of a fatal error. */ private final FatalErrorHandler fatalErrorHandler; @@ -207,10 +205,10 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { private FileCache fileCache; /** The heartbeat manager for job manager in the task manager. */ - private HeartbeatManager jobManagerHeartbeatManager; + private final HeartbeatManager jobManagerHeartbeatManager; /** The heartbeat manager for resource manager in the task manager. */ - private HeartbeatManager resourceManagerHeartbeatManager; + private final HeartbeatManager resourceManagerHeartbeatManager; private final PartitionTable partitionTable; @@ -249,7 +247,6 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { checkArgument(taskManagerConfiguration.getNumberSlots() > 0, "The number of slots has to be larger than 0."); this.taskManagerConfiguration = checkNotNull(taskManagerConfiguration); - this.heartbeatServices = checkNotNull(heartbeatServices); this.taskExecutorServices = checkNotNull(taskExecutorServices); this.haServices = checkNotNull(haServices); this.fatalErrorHandler = checkNotNull(fatalErrorHandler); @@ -278,6 +275,26 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { this.stackTraceSampleService = new StackTraceSampleService(rpcService.getScheduledExecutor()); this.taskCompletionTracker = new TaskCompletionTracker(); + + final ResourceID resourceId = taskExecutorServices.getTaskManagerLocation().getResourceID(); + this.jobManagerHeartbeatManager = createJobManagerHeartbeatManager(heartbeatServices, resourceId); + this.resourceManagerHeartbeatManager = createResourceManagerHeartbeatManager(heartbeatServices, resourceId); + } + + private HeartbeatManager createResourceManagerHeartbeatManager(HeartbeatServices heartbeatServices, ResourceID resourceId) { + return heartbeatServices.createHeartbeatManager( + resourceId, + new ResourceManagerHeartbeatListener(), + getMainThreadExecutor(), + log); + } + + private HeartbeatManager createJobManagerHeartbeatManager(HeartbeatServices heartbeatServices, ResourceID resourceId) { + return heartbeatServices.createHeartbeatManager( + resourceId, + new JobManagerHeartbeatListener(), + getMainThreadExecutor(), + log); } @Override @@ -304,8 +321,6 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { private void startTaskExecutorServices() throws Exception { try { - startHeartbeatServices(); - // start by connecting to the ResourceManager resourceManagerLeaderRetriever.start(new ResourceManagerLeaderListener()); @@ -412,38 +427,9 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { // it will call close() recursively from the parent to children taskManagerMetricGroup.close(); - stopHeartbeatServices(); - ExceptionUtils.tryRethrowException(exception); } - private void startHeartbeatServices() { - final ResourceID resourceId = taskExecutorServices.getTaskManagerLocation().getResourceID(); - jobManagerHeartbeatManager = heartbeatServices.createHeartbeatManager( - resourceId, - new JobManagerHeartbeatListener(), - getMainThreadExecutor(), - log); - - resourceManagerHeartbeatManager = heartbeatServices.createHeartbeatManager( - resourceId, - new ResourceManagerHeartbeatListener(), - getMainThreadExecutor(), - log); - } - - private void stopHeartbeatServices() { - if (jobManagerHeartbeatManager != null) { - jobManagerHeartbeatManager.stop(); - jobManagerHeartbeatManager = null; - } - - if (resourceManagerHeartbeatManager != null) { - resourceManagerHeartbeatManager.stop(); - resourceManagerHeartbeatManager = null; - } - } - // ====================================================================== // RPC methods // ====================================================================== -- Gitee From be8fdcaea45136b15e8783831ac3d2bdb2608c3b Mon Sep 17 00:00:00 2001 From: ouyangwulin Date: Tue, 8 Oct 2019 22:50:40 +0800 Subject: [PATCH 154/268] [FLINK-14210][metrics][influxdb] Make timeouts configurable --- docs/monitoring/metrics.md | 4 ++++ docs/monitoring/metrics.zh.md | 4 ++++ .../flink/metrics/influxdb/InfluxdbReporter.java | 15 +++++++++++++-- .../metrics/influxdb/InfluxdbReporterOptions.java | 10 ++++++++++ 4 files changed, 31 insertions(+), 2 deletions(-) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index 628184d256..70fdcc5cf3 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -659,6 +659,8 @@ Parameters: - `username` - (optional) InfluxDB username used for authentication - `password` - (optional) InfluxDB username's password used for authentication - `retentionPolicy` - (optional) InfluxDB retention policy, defaults to retention policy defined on the server for the db +- `connectTimeout` - (optional) the InfluxDB client connect timeout in milliseconds, default is 10000 ms +- `writeTimeout` - (optional) the InfluxDB client write timeout in milliseconds, default is 10000 ms Example configuration: @@ -671,6 +673,8 @@ metrics.reporter.influxdb.db: flink metrics.reporter.influxdb.username: flink-metrics metrics.reporter.influxdb.password: qwerty metrics.reporter.influxdb.retentionPolicy: one_hour +metrics.reporter.influxdb.connectTimeout: 60000 +metrics.reporter.influxdb.writeTimeout: 60000 {% endhighlight %} diff --git a/docs/monitoring/metrics.zh.md b/docs/monitoring/metrics.zh.md index 6b0dbf1b6f..fa8e14e2f8 100644 --- a/docs/monitoring/metrics.zh.md +++ b/docs/monitoring/metrics.zh.md @@ -659,6 +659,8 @@ Parameters: - `username` - (optional) InfluxDB username used for authentication - `password` - (optional) InfluxDB username's password used for authentication - `retentionPolicy` - (optional) InfluxDB retention policy, defaults to retention policy defined on the server for the db +- `connectTimeout` - (optional) the InfluxDB client connect timeout in milliseconds, default is 10000 ms +- `writeTimeout` - (optional) the InfluxDB client write timeout in milliseconds, default is 10000 ms Example configuration: @@ -671,6 +673,8 @@ metrics.reporter.influxdb.db: flink metrics.reporter.influxdb.username: flink-metrics metrics.reporter.influxdb.password: qwerty metrics.reporter.influxdb.retentionPolicy: one_hour +metrics.reporter.influxdb.connectTimeout: 60000 +metrics.reporter.influxdb.writeTimeout: 60000 {% endhighlight %} diff --git a/flink-metrics/flink-metrics-influxdb/src/main/java/org/apache/flink/metrics/influxdb/InfluxdbReporter.java b/flink-metrics/flink-metrics-influxdb/src/main/java/org/apache/flink/metrics/influxdb/InfluxdbReporter.java index 28d9eee184..19404ffbe0 100644 --- a/flink-metrics/flink-metrics-influxdb/src/main/java/org/apache/flink/metrics/influxdb/InfluxdbReporter.java +++ b/flink-metrics/flink-metrics-influxdb/src/main/java/org/apache/flink/metrics/influxdb/InfluxdbReporter.java @@ -27,6 +27,7 @@ import org.apache.flink.metrics.MetricConfig; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.reporter.Scheduled; +import okhttp3.OkHttpClient; import org.influxdb.InfluxDB; import org.influxdb.InfluxDBFactory; import org.influxdb.dto.BatchPoints; @@ -37,13 +38,16 @@ import java.time.Instant; import java.util.ConcurrentModificationException; import java.util.Map; import java.util.NoSuchElementException; +import java.util.concurrent.TimeUnit; +import static org.apache.flink.metrics.influxdb.InfluxdbReporterOptions.CONNECT_TIMEOUT; import static org.apache.flink.metrics.influxdb.InfluxdbReporterOptions.DB; import static org.apache.flink.metrics.influxdb.InfluxdbReporterOptions.HOST; import static org.apache.flink.metrics.influxdb.InfluxdbReporterOptions.PASSWORD; import static org.apache.flink.metrics.influxdb.InfluxdbReporterOptions.PORT; import static org.apache.flink.metrics.influxdb.InfluxdbReporterOptions.RETENTION_POLICY; import static org.apache.flink.metrics.influxdb.InfluxdbReporterOptions.USERNAME; +import static org.apache.flink.metrics.influxdb.InfluxdbReporterOptions.WRITE_TIMEOUT; import static org.apache.flink.metrics.influxdb.InfluxdbReporterOptions.getInteger; import static org.apache.flink.metrics.influxdb.InfluxdbReporterOptions.getString; @@ -77,10 +81,17 @@ public class InfluxdbReporter extends AbstractReporter implemen this.database = database; this.retentionPolicy = getString(config, RETENTION_POLICY); + + int connectTimeout = getInteger(config, CONNECT_TIMEOUT); + int writeTimeout = getInteger(config, WRITE_TIMEOUT); + OkHttpClient.Builder client = new OkHttpClient.Builder() + .connectTimeout(connectTimeout, TimeUnit.MILLISECONDS) + .writeTimeout(writeTimeout, TimeUnit.MILLISECONDS); + if (username != null && password != null) { - influxDB = InfluxDBFactory.connect(url, username, password); + influxDB = InfluxDBFactory.connect(url, username, password, client); } else { - influxDB = InfluxDBFactory.connect(url); + influxDB = InfluxDBFactory.connect(url, client); } log.info("Configured InfluxDBReporter with {host:{}, port:{}, db:{}, and retentionPolicy:{}}", host, port, database, retentionPolicy); diff --git a/flink-metrics/flink-metrics-influxdb/src/main/java/org/apache/flink/metrics/influxdb/InfluxdbReporterOptions.java b/flink-metrics/flink-metrics-influxdb/src/main/java/org/apache/flink/metrics/influxdb/InfluxdbReporterOptions.java index fe21fb083c..cc88eb5f33 100644 --- a/flink-metrics/flink-metrics-influxdb/src/main/java/org/apache/flink/metrics/influxdb/InfluxdbReporterOptions.java +++ b/flink-metrics/flink-metrics-influxdb/src/main/java/org/apache/flink/metrics/influxdb/InfluxdbReporterOptions.java @@ -57,6 +57,16 @@ public class InfluxdbReporterOptions { .defaultValue("") .withDescription("(optional) the InfluxDB retention policy for metrics"); + public static final ConfigOption CONNECT_TIMEOUT = ConfigOptions + .key("connectTimeout") + .defaultValue(10000) + .withDescription("(optional) the InfluxDB connect timeout for metrics"); + + public static final ConfigOption WRITE_TIMEOUT = ConfigOptions + .key("writeTimeout") + .defaultValue(10000) + .withDescription("(optional) the InfluxDB write timeout for metrics"); + static String getString(MetricConfig config, ConfigOption key) { return config.getString(key.key(), key.defaultValue()); } -- Gitee From 71dfb4cbc337261f4a799954128df46d685542d9 Mon Sep 17 00:00:00 2001 From: Yangze Guo Date: Wed, 9 Oct 2019 16:26:51 +0800 Subject: [PATCH 155/268] [FLINK-14335][docs] Fix ExampleIntegrationTest example - java version wasn't compiling due to missing ';' - java version was checking order of the elements which cannot be guaranteed - examples where checking for the wrong results --- docs/dev/stream/testing.md | 12 ++++++------ docs/dev/stream/testing.zh.md | 12 ++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/docs/dev/stream/testing.md b/docs/dev/stream/testing.md index 8992d50950..2bd0096bbf 100644 --- a/docs/dev/stream/testing.md +++ b/docs/dev/stream/testing.md @@ -44,7 +44,7 @@ public class IncrementMapFunction implements MapFunction { @Override public Long map(Long record) throws Exception { - return record +1 ; + return record + 1; } } {% endhighlight %} @@ -112,7 +112,7 @@ public class IncrementFlatMapFunctionTest { Collector collector = mock(Collector.class); // call the methods that you have implemented - incrementer.flatMap(2L, collector) + incrementer.flatMap(2L, collector); //verify collector was called with the right output Mockito.verify(collector, times(1)).collect(3L); @@ -216,7 +216,7 @@ public class StatefulFlatMapTest { testHarness.setProcessingTime(100L); //retrieve list of emitted records for assertions - assertThat(testHarness.getOutput(), containsInExactlyThisOrder(3L)) + assertThat(testHarness.getOutput(), containsInExactlyThisOrder(3L)); //retrieve list of records emitted to a specific side output for assertions (ProcessFunction only) //assertThat(testHarness.getSideOutput(new OutputTag<>("invalidRecords")), hasSize(0)) @@ -358,7 +358,7 @@ public class IncrementMapFunction implements MapFunction { @Override public Long map(Long record) throws Exception { - return record +1 ; + return record + 1; } } {% endhighlight %} @@ -410,7 +410,7 @@ public class ExampleIntegrationTest { env.execute(); // verify your results - assertEquals(Lists.newArrayList(2L, 42L, 44L), CollectSink.values); + assertTrue(CollectSink.values.containsAll(2L, 22L, 23L)); } // create a testing sink @@ -465,7 +465,7 @@ class StreamingJobIntegrationTest extends FlatSpec with Matchers with BeforeAndA env.execute() // verify your results - CollectSink.values should contain allOf (1,22,23) + CollectSink.values should contain allOf (2, 22, 23) } } // create a testing sink diff --git a/docs/dev/stream/testing.zh.md b/docs/dev/stream/testing.zh.md index 01c780895c..2264ec40cd 100644 --- a/docs/dev/stream/testing.zh.md +++ b/docs/dev/stream/testing.zh.md @@ -44,7 +44,7 @@ public class IncrementMapFunction implements MapFunction { @Override public Long map(Long record) throws Exception { - return record +1 ; + return record + 1; } } {% endhighlight %} @@ -112,7 +112,7 @@ public class IncrementFlatMapFunctionTest { Collector collector = mock(Collector.class); // call the methods that you have implemented - incrementer.flatMap(2L, collector) + incrementer.flatMap(2L, collector); //verify collector was called with the right output Mockito.verify(collector, times(1)).collect(3L); @@ -216,7 +216,7 @@ public class StatefulFlatMapTest { testHarness.setProcessingTime(100L); //retrieve list of emitted records for assertions - assertThat(testHarness.getOutput(), containsInExactlyThisOrder(3L)) + assertThat(testHarness.getOutput(), containsInExactlyThisOrder(3L)); //retrieve list of records emitted to a specific side output for assertions (ProcessFunction only) //assertThat(testHarness.getSideOutput(new OutputTag<>("invalidRecords")), hasSize(0)) @@ -358,7 +358,7 @@ public class IncrementMapFunction implements MapFunction { @Override public Long map(Long record) throws Exception { - return record +1 ; + return record + 1; } } {% endhighlight %} @@ -410,7 +410,7 @@ public class ExampleIntegrationTest { env.execute(); // verify your results - assertEquals(Lists.newArrayList(2L, 42L, 44L), CollectSink.values); + assertTrue(CollectSink.values.containsAll(2L, 22L, 23L)); } // create a testing sink @@ -465,7 +465,7 @@ class StreamingJobIntegrationTest extends FlatSpec with Matchers with BeforeAndA env.execute() // verify your results - CollectSink.values should contain allOf (1,22,23) + CollectSink.values should contain allOf (2, 22, 23) } } // create a testing sink -- Gitee From c910d71c358ab55763e77fdd2be2811dcdec6c41 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 9 Oct 2019 15:25:13 +0200 Subject: [PATCH 156/268] [hotfix] Update README - add Java 11 to list of supported Java versions - add WSL to list of unix-like environments - updated some links to use https --- README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index c67526cbc6..9733b3df75 100644 --- a/README.md +++ b/README.md @@ -67,10 +67,10 @@ counts.writeAsCsv(outputPath) Prerequisites for building Flink: -* Unix-like environment (we use Linux, Mac OS X, Cygwin) +* Unix-like environment (we use Linux, Mac OS X, Cygwin, WSL) * Git * Maven (we recommend version 3.2.5 and require at least 3.1.1) -* Java 8 (Java 9 and 10 are not yet supported) +* Java 8 or 11 (Java 9 or 10 may work) ``` git clone https://github.com/apache/flink.git @@ -78,7 +78,7 @@ cd flink mvn clean package -DskipTests # this will take up to 10 minutes ``` -Flink is now installed in `build-target` +Flink is now installed in `build-target`. *NOTE: Maven 3.3.x can build Flink, but will not properly shade away certain dependencies. Maven 3.1.1 creates the libraries properly. To build unit tests with Java 8, use Java 8u51 or above to prevent failures in unit tests that use the PowerMock runner.* @@ -98,7 +98,7 @@ Minimal requirements for an IDE are: The IntelliJ IDE supports Maven out of the box and offers a plugin for Scala development. * 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) +* IntelliJ Scala Plugin: [https://plugins.jetbrains.com/plugin/?id=1347](https://plugins.jetbrains.com/plugin/?id=1347) Check out our [Setting up IntelliJ](https://ci.apache.org/projects/flink/flink-docs-master/flinkDev/ide_setup.html#intellij-idea) guide for details. -- Gitee From 0d112f5bc61e6f8400e000e13add08abae1067a1 Mon Sep 17 00:00:00 2001 From: Marcos Klein Date: Wed, 9 Oct 2019 06:56:46 -0700 Subject: [PATCH 157/268] [FLINK-14300][runtime] Cleanup operator threads in case StreamTask fails to allocate operatorChain (#9857) This commit fixes a thread leak on the task manager when the StreamTask class fails to deserialize an operator when instantiating the operatorChain property. The error handling code cleans up all operator threads if the OperatorChain class is instantiated, but fails to clean up threads created before the OperatorChain class is instantiated if the operatorChain property is null. An example of a deserialization exception thrown when instantiating the OperatorChain object is as follows: org.apache.flink.streaming.runtime.tasks.StreamTaskException: Cannot instantiate user function. at org.apache.flink.streaming.api.graph.StreamConfig.getStreamOperator(StreamConfig.java:239) at org.apache.flink.streaming.runtime.tasks.OperatorChain.(OperatorChain.java:104) at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:267) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711) at java.lang.Thread.run(Thread.java:748) Caused by: java.io.InvalidClassException: (...) --- .../org/apache/flink/streaming/runtime/tasks/StreamTask.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 1617ac7fe2..951fbf9678 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -514,6 +514,11 @@ public abstract class StreamTask> synchronized (lock) { operatorChain.releaseOutputs(); } + } else { + // failed to allocate operatorChain, clean up record writers + for (RecordWriter>> writer: recordWriters) { + writer.close(); + } } mailboxProcessor.close(); -- Gitee From df44cce7f1e1bc42715e6cd0e7268c1a46b07af8 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Tue, 8 Oct 2019 20:20:53 +0800 Subject: [PATCH 158/268] [FLINK-14306][python] Add the code-generated flink_fn_execution_pb2.py to the source code Previously, we generate some python file during the building. However, it introduces python dependencies which makes the building complicated. This commit adds the generated file directly into the source code manually so that there is no python dependencies during building. This closes #9855. --- .gitignore | 1 - flink-python/README.md | 13 + flink-python/pom.xml | 20 - .../fn_execution/flink_fn_execution_pb2.py | 509 ++++++++++++++++++ .../test_flink_fn_execution_pb2_synced.py | 43 ++ flink-python/pyflink/gen_protos.py | 70 ++- .../pyflink/proto/flink-fn-execution.proto | 2 + .../pyflink/testing/test_case_utils.py | 3 - flink-python/setup.py | 36 +- pom.xml | 1 - 10 files changed, 623 insertions(+), 75 deletions(-) create mode 100644 flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py create mode 100644 flink-python/pyflink/fn_execution/tests/test_flink_fn_execution_pb2_synced.py diff --git a/.gitignore b/.gitignore index 4e3a5ae2f2..e89f6224ee 100644 --- a/.gitignore +++ b/.gitignore @@ -36,7 +36,6 @@ flink-python/dev/.conda/ flink-python/dev/log/ flink-python/dev/.stage.txt flink-python/.eggs/ -flink-python/pyflink/fn_execution/*_pb2.py atlassian-ide-plugin.xml out/ /docs/api diff --git a/flink-python/README.md b/flink-python/README.md index db49150381..76df93dd77 100644 --- a/flink-python/README.md +++ b/flink-python/README.md @@ -28,3 +28,16 @@ We can enter the directory where this README.md file is located and run test cas ## Python Requirements PyFlink depends on Py4J (currently version 0.10.8.1) and CloudPickle (currently version 1.2.2). + +## Development notices + +Protocol buffer is used in this module and file `flink_fn_execution_pb2.py` is generated from `flink-fn-execution.proto`. Whenever `flink-fn-execution.proto` is updated, please re-generate `flink_fn_execution_pb2.py` by executing + +``` +python pyflink/gen_protos.py +``` + +PyFlink depends on the following libraries to execute the above script: +1. grpcio-tools (>=1.3.5,<=1.14.2) +2. setuptools (>=37.0.0) +3. pip (>=8.0.0) diff --git a/flink-python/pom.xml b/flink-python/pom.xml index ffeb4864c5..ed16daa09e 100644 --- a/flink-python/pom.xml +++ b/flink-python/pom.xml @@ -327,26 +327,6 @@ under the License.
- - exec-maven-plugin - org.codehaus.mojo - 1.5.0 - - - Protos Generation - generate-sources - - exec - - - python - - ${basedir}/pyflink/gen_protos.py - - - - - diff --git a/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py new file mode 100644 index 0000000000..b473673460 --- /dev/null +++ b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py @@ -0,0 +1,509 @@ +################################################################################ +# 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. +################################################################################ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: flink-fn-execution.proto + +import sys +_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +from google.protobuf import descriptor_pb2 +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='flink-fn-execution.proto', + package='org.apache.flink.fn_execution.v1', + syntax='proto3', + serialized_pb=_b('\n\x18\x66link-fn-execution.proto\x12 org.apache.flink.fn_execution.v1\"\xe2\x01\n\x13UserDefinedFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12K\n\x06inputs\x18\x02 \x03(\x0b\x32;.org.apache.flink.fn_execution.v1.UserDefinedFunction.Input\x1am\n\x05Input\x12\x44\n\x03udf\x18\x01 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunctionH\x00\x12\x15\n\x0binputOffset\x18\x02 \x01(\x05H\x00\x42\x07\n\x05input\"[\n\x14UserDefinedFunctions\x12\x43\n\x04udfs\x18\x01 \x03(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunction\"\x8d\x07\n\x06Schema\x12>\n\x06\x66ields\x18\x01 \x03(\x0b\x32..org.apache.flink.fn_execution.v1.Schema.Field\x1a\x97\x01\n\x07MapType\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\xcd\x02\n\tFieldType\x12\x44\n\ttype_name\x18\x01 \x01(\x0e\x32\x31.org.apache.flink.fn_execution.v1.Schema.TypeName\x12\x10\n\x08nullable\x18\x02 \x01(\x08\x12U\n\x17\x63ollection_element_type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeH\x00\x12\x44\n\x08map_type\x18\x04 \x01(\x0b\x32\x30.org.apache.flink.fn_execution.v1.Schema.MapTypeH\x00\x12>\n\nrow_schema\x18\x05 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x42\x0b\n\ttype_info\x1al\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12@\n\x04type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\"\xea\x01\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\x0b\n\x07TINYINT\x10\x01\x12\x0c\n\x08SMALLINT\x10\x02\x12\x07\n\x03INT\x10\x03\x12\n\n\x06\x42IGINT\x10\x04\x12\x0b\n\x07\x44\x45\x43IMAL\x10\x05\x12\t\n\x05\x46LOAT\x10\x06\x12\n\n\x06\x44OUBLE\x10\x07\x12\x08\n\x04\x44\x41TE\x10\x08\x12\x08\n\x04TIME\x10\t\x12\x0c\n\x08\x44\x41TETIME\x10\n\x12\x0b\n\x07\x42OOLEAN\x10\x0b\x12\n\n\x06\x42INARY\x10\x0c\x12\r\n\tVARBINARY\x10\r\x12\x08\n\x04\x43HAR\x10\x0e\x12\x0b\n\x07VARCHAR\x10\x0f\x12\t\n\x05\x41RRAY\x10\x10\x12\x07\n\x03MAP\x10\x11\x12\x0c\n\x08MULTISET\x10\x12\x42-\n\x1forg.apache.flink.fnexecution.v1B\nFlinkFnApib\x06proto3') +) + + + +_SCHEMA_TYPENAME = _descriptor.EnumDescriptor( + name='TypeName', + full_name='org.apache.flink.fn_execution.v1.Schema.TypeName', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='ROW', index=0, number=0, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='TINYINT', index=1, number=1, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='SMALLINT', index=2, number=2, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='INT', index=3, number=3, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='BIGINT', index=4, number=4, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='DECIMAL', index=5, number=5, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='FLOAT', index=6, number=6, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='DOUBLE', index=7, number=7, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='DATE', index=8, number=8, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='TIME', index=9, number=9, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='DATETIME', index=10, number=10, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='BOOLEAN', index=11, number=11, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='BINARY', index=12, number=12, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='VARBINARY', index=13, number=13, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='CHAR', index=14, number=14, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='VARCHAR', index=15, number=15, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='ARRAY', index=16, number=16, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='MAP', index=17, number=17, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='MULTISET', index=18, number=18, + options=None, + type=None), + ], + containing_type=None, + options=None, + serialized_start=1060, + serialized_end=1294, +) +_sym_db.RegisterEnumDescriptor(_SCHEMA_TYPENAME) + + +_USERDEFINEDFUNCTION_INPUT = _descriptor.Descriptor( + name='Input', + full_name='org.apache.flink.fn_execution.v1.UserDefinedFunction.Input', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='udf', full_name='org.apache.flink.fn_execution.v1.UserDefinedFunction.Input.udf', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='inputOffset', full_name='org.apache.flink.fn_execution.v1.UserDefinedFunction.Input.inputOffset', index=1, + number=2, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + _descriptor.OneofDescriptor( + name='input', full_name='org.apache.flink.fn_execution.v1.UserDefinedFunction.Input.input', + index=0, containing_type=None, fields=[]), + ], + serialized_start=180, + serialized_end=289, +) + +_USERDEFINEDFUNCTION = _descriptor.Descriptor( + name='UserDefinedFunction', + full_name='org.apache.flink.fn_execution.v1.UserDefinedFunction', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='payload', full_name='org.apache.flink.fn_execution.v1.UserDefinedFunction.payload', index=0, + number=1, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='inputs', full_name='org.apache.flink.fn_execution.v1.UserDefinedFunction.inputs', index=1, + number=2, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[_USERDEFINEDFUNCTION_INPUT, ], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=63, + serialized_end=289, +) + + +_USERDEFINEDFUNCTIONS = _descriptor.Descriptor( + name='UserDefinedFunctions', + full_name='org.apache.flink.fn_execution.v1.UserDefinedFunctions', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='udfs', full_name='org.apache.flink.fn_execution.v1.UserDefinedFunctions.udfs', index=0, + number=1, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=291, + serialized_end=382, +) + + +_SCHEMA_MAPTYPE = _descriptor.Descriptor( + name='MapType', + full_name='org.apache.flink.fn_execution.v1.Schema.MapType', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key_type', full_name='org.apache.flink.fn_execution.v1.Schema.MapType.key_type', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='value_type', full_name='org.apache.flink.fn_execution.v1.Schema.MapType.value_type', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=460, + serialized_end=611, +) + +_SCHEMA_FIELDTYPE = _descriptor.Descriptor( + name='FieldType', + full_name='org.apache.flink.fn_execution.v1.Schema.FieldType', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='type_name', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.type_name', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='nullable', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.nullable', index=1, + number=2, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='collection_element_type', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.collection_element_type', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='map_type', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.map_type', index=3, + number=4, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='row_schema', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.row_schema', index=4, + number=5, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + _descriptor.OneofDescriptor( + name='type_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.type_info', + index=0, containing_type=None, fields=[]), + ], + serialized_start=614, + serialized_end=947, +) + +_SCHEMA_FIELD = _descriptor.Descriptor( + name='Field', + full_name='org.apache.flink.fn_execution.v1.Schema.Field', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='name', full_name='org.apache.flink.fn_execution.v1.Schema.Field.name', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='description', full_name='org.apache.flink.fn_execution.v1.Schema.Field.description', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='type', full_name='org.apache.flink.fn_execution.v1.Schema.Field.type', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=949, + serialized_end=1057, +) + +_SCHEMA = _descriptor.Descriptor( + name='Schema', + full_name='org.apache.flink.fn_execution.v1.Schema', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='fields', full_name='org.apache.flink.fn_execution.v1.Schema.fields', index=0, + number=1, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[_SCHEMA_MAPTYPE, _SCHEMA_FIELDTYPE, _SCHEMA_FIELD, ], + enum_types=[ + _SCHEMA_TYPENAME, + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=385, + serialized_end=1294, +) + +_USERDEFINEDFUNCTION_INPUT.fields_by_name['udf'].message_type = _USERDEFINEDFUNCTION +_USERDEFINEDFUNCTION_INPUT.containing_type = _USERDEFINEDFUNCTION +_USERDEFINEDFUNCTION_INPUT.oneofs_by_name['input'].fields.append( + _USERDEFINEDFUNCTION_INPUT.fields_by_name['udf']) +_USERDEFINEDFUNCTION_INPUT.fields_by_name['udf'].containing_oneof = _USERDEFINEDFUNCTION_INPUT.oneofs_by_name['input'] +_USERDEFINEDFUNCTION_INPUT.oneofs_by_name['input'].fields.append( + _USERDEFINEDFUNCTION_INPUT.fields_by_name['inputOffset']) +_USERDEFINEDFUNCTION_INPUT.fields_by_name['inputOffset'].containing_oneof = _USERDEFINEDFUNCTION_INPUT.oneofs_by_name['input'] +_USERDEFINEDFUNCTION.fields_by_name['inputs'].message_type = _USERDEFINEDFUNCTION_INPUT +_USERDEFINEDFUNCTIONS.fields_by_name['udfs'].message_type = _USERDEFINEDFUNCTION +_SCHEMA_MAPTYPE.fields_by_name['key_type'].message_type = _SCHEMA_FIELDTYPE +_SCHEMA_MAPTYPE.fields_by_name['value_type'].message_type = _SCHEMA_FIELDTYPE +_SCHEMA_MAPTYPE.containing_type = _SCHEMA +_SCHEMA_FIELDTYPE.fields_by_name['type_name'].enum_type = _SCHEMA_TYPENAME +_SCHEMA_FIELDTYPE.fields_by_name['collection_element_type'].message_type = _SCHEMA_FIELDTYPE +_SCHEMA_FIELDTYPE.fields_by_name['map_type'].message_type = _SCHEMA_MAPTYPE +_SCHEMA_FIELDTYPE.fields_by_name['row_schema'].message_type = _SCHEMA +_SCHEMA_FIELDTYPE.containing_type = _SCHEMA +_SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( + _SCHEMA_FIELDTYPE.fields_by_name['collection_element_type']) +_SCHEMA_FIELDTYPE.fields_by_name['collection_element_type'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] +_SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( + _SCHEMA_FIELDTYPE.fields_by_name['map_type']) +_SCHEMA_FIELDTYPE.fields_by_name['map_type'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] +_SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( + _SCHEMA_FIELDTYPE.fields_by_name['row_schema']) +_SCHEMA_FIELDTYPE.fields_by_name['row_schema'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] +_SCHEMA_FIELD.fields_by_name['type'].message_type = _SCHEMA_FIELDTYPE +_SCHEMA_FIELD.containing_type = _SCHEMA +_SCHEMA.fields_by_name['fields'].message_type = _SCHEMA_FIELD +_SCHEMA_TYPENAME.containing_type = _SCHEMA +DESCRIPTOR.message_types_by_name['UserDefinedFunction'] = _USERDEFINEDFUNCTION +DESCRIPTOR.message_types_by_name['UserDefinedFunctions'] = _USERDEFINEDFUNCTIONS +DESCRIPTOR.message_types_by_name['Schema'] = _SCHEMA +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + +UserDefinedFunction = _reflection.GeneratedProtocolMessageType('UserDefinedFunction', (_message.Message,), dict( + + Input = _reflection.GeneratedProtocolMessageType('Input', (_message.Message,), dict( + DESCRIPTOR = _USERDEFINEDFUNCTION_INPUT, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.UserDefinedFunction.Input) + )) + , + DESCRIPTOR = _USERDEFINEDFUNCTION, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.UserDefinedFunction) + )) +_sym_db.RegisterMessage(UserDefinedFunction) +_sym_db.RegisterMessage(UserDefinedFunction.Input) + +UserDefinedFunctions = _reflection.GeneratedProtocolMessageType('UserDefinedFunctions', (_message.Message,), dict( + DESCRIPTOR = _USERDEFINEDFUNCTIONS, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.UserDefinedFunctions) + )) +_sym_db.RegisterMessage(UserDefinedFunctions) + +Schema = _reflection.GeneratedProtocolMessageType('Schema', (_message.Message,), dict( + + MapType = _reflection.GeneratedProtocolMessageType('MapType', (_message.Message,), dict( + DESCRIPTOR = _SCHEMA_MAPTYPE, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.MapType) + )) + , + + FieldType = _reflection.GeneratedProtocolMessageType('FieldType', (_message.Message,), dict( + DESCRIPTOR = _SCHEMA_FIELDTYPE, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.FieldType) + )) + , + + Field = _reflection.GeneratedProtocolMessageType('Field', (_message.Message,), dict( + DESCRIPTOR = _SCHEMA_FIELD, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.Field) + )) + , + DESCRIPTOR = _SCHEMA, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema) + )) +_sym_db.RegisterMessage(Schema) +_sym_db.RegisterMessage(Schema.MapType) +_sym_db.RegisterMessage(Schema.FieldType) +_sym_db.RegisterMessage(Schema.Field) + + +DESCRIPTOR.has_options = True +DESCRIPTOR._options = _descriptor._ParseOptions(descriptor_pb2.FileOptions(), _b('\n\037org.apache.flink.fnexecution.v1B\nFlinkFnApi')) +# @@protoc_insertion_point(module_scope) diff --git a/flink-python/pyflink/fn_execution/tests/test_flink_fn_execution_pb2_synced.py b/flink-python/pyflink/fn_execution/tests/test_flink_fn_execution_pb2_synced.py new file mode 100644 index 0000000000..544e4f3a70 --- /dev/null +++ b/flink-python/pyflink/fn_execution/tests/test_flink_fn_execution_pb2_synced.py @@ -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. +################################################################################ +import filecmp +import os + +from pyflink.gen_protos import generate_proto_files +from pyflink.testing.test_case_utils import PyFlinkTestCase + + +class FlinkFnExecutionSyncTests(PyFlinkTestCase): + """ + Tests whether flink_fn_exeution_pb2.py is synced with flink-fn-execution.proto. + """ + + flink_fn_execution_pb2_file_name = "flink_fn_execution_pb2.py" + gen_protos_script = "gen_protos.py" + flink_fn_execution_proto_file_name = "flink-fn-execution.proto" + + def test_flink_fn_execution_pb2_synced(self): + generate_proto_files('True', self.tempdir) + expected = os.path.join(self.tempdir, self.flink_fn_execution_pb2_file_name) + actual = os.path.join(os.path.dirname(os.path.abspath(__file__)), '..', + self.flink_fn_execution_pb2_file_name) + self.assertTrue(filecmp.cmp(expected, actual), + 'File %s should be re-generated by executing %s as %s has changed.' + % (self.flink_fn_execution_pb2_file_name, + self.gen_protos_script, + self.flink_fn_execution_proto_file_name)) diff --git a/flink-python/pyflink/gen_protos.py b/flink-python/pyflink/gen_protos.py index ce4ddd4010..de54d5fb05 100644 --- a/flink-python/pyflink/gen_protos.py +++ b/flink-python/pyflink/gen_protos.py @@ -34,25 +34,21 @@ import pkg_resources # latest grpcio-tools incompatible with latest protobuf 3.6.1. GRPC_TOOLS = 'grpcio-tools>=1.3.5,<=1.14.2' +PROTO_PATHS = ['proto'] +PYFLINK_ROOT_PATH = os.path.dirname(os.path.abspath(__file__)) +DEFAULT_PYTHON_OUTPUT_PATH = os.path.join(PYFLINK_ROOT_PATH, 'fn_execution') -PROTO_PATHS = [ - os.path.join('proto'), -] -PYTHON_OUTPUT_PATH = os.path.join('fn_execution') - - -def generate_proto_files(force=False): +def generate_proto_files(force=True, output_dir=DEFAULT_PYTHON_OUTPUT_PATH): try: import grpc_tools # noqa # pylint: disable=unused-import except ImportError: warnings.warn('Installing grpcio-tools is recommended for development.') - py_sdk_root = os.path.dirname(os.path.abspath(__file__)) - proto_dirs = [os.path.join(py_sdk_root, path) for path in PROTO_PATHS] + proto_dirs = [os.path.join(PYFLINK_ROOT_PATH, path) for path in PROTO_PATHS] proto_files = sum( [glob.glob(os.path.join(d, '*.proto')) for d in proto_dirs], []) - out_dir = os.path.join(py_sdk_root, PYTHON_OUTPUT_PATH) + out_dir = os.path.join(PYFLINK_ROOT_PATH, output_dir) out_files = [path for path in glob.glob(os.path.join(out_dir, '*_pb2.py'))] if out_files and not proto_files and not force: @@ -84,12 +80,13 @@ def generate_proto_files(force=False): # Note that this requires a separate module from setup.py for Windows: # https://docs.python.org/2/library/multiprocessing.html#windows p = multiprocessing.Process( - target=_install_grpcio_tools_and_generate_proto_files) + target=_install_grpcio_tools_and_generate_proto_files(force, output_dir)) p.start() p.join() if p.exitcode: raise ValueError("Proto generation failed (see log for details).") else: + _check_grpcio_tools_version() logging.info('Regenerating out-of-date Python proto definitions.') builtin_protos = pkg_resources.resource_filename('grpc_tools', '_proto') args = ( @@ -104,6 +101,10 @@ def generate_proto_files(force=False): 'Protoc returned non-zero status (see logs for details): ' '%s' % ret_code) + for output_file in os.listdir(output_dir): + if output_file.endswith('_pb2.py'): + _add_license_header(output_dir, output_file) + # Though wheels are available for grpcio-tools, setup_requires uses # easy_install which doesn't understand them. This means that it is @@ -111,9 +112,8 @@ def generate_proto_files(force=False): # protoc compiler). Instead, we attempt to install a wheel in a temporary # directory and add it to the path as needed. # See https://github.com/pypa/setuptools/issues/377 -def _install_grpcio_tools_and_generate_proto_files(): - install_path = os.path.join( - os.path.dirname(os.path.abspath(__file__)), '..', '.eggs', 'grpcio-wheels') +def _install_grpcio_tools_and_generate_proto_files(force, output_dir): + install_path = os.path.join(PYFLINK_ROOT_PATH, '..', '.eggs', 'grpcio-wheels') build_path = install_path + '-build' if os.path.exists(build_path): shutil.rmtree(build_path) @@ -134,13 +134,51 @@ def _install_grpcio_tools_and_generate_proto_files(): sys.stderr.flush() shutil.rmtree(build_path, ignore_errors=True) sys.path.append(install_obj.install_purelib) + pkg_resources.working_set.add_entry(install_obj.install_purelib) if install_obj.install_purelib != install_obj.install_platlib: sys.path.append(install_obj.install_platlib) + pkg_resources.working_set.add_entry(install_obj.install_platlib) try: - generate_proto_files() + generate_proto_files(force, output_dir) finally: sys.stderr.flush() +def _add_license_header(dir, file_name): + with open(os.path.join(dir, file_name), 'r') as original_file: + original_data = original_file.read() + tmp_file_name = file_name + '.tmp' + with open(os.path.join(dir, tmp_file_name), 'w') as tmp_file: + tmp_file.write( + '################################################################################\n' + '# Licensed to the Apache Software Foundation (ASF) under one\n' + '# or more contributor license agreements. See the NOTICE file\n' + '# distributed with this work for additional information\n' + '# regarding copyright ownership. The ASF licenses this file\n' + '# to you under the Apache License, Version 2.0 (the\n' + '# "License"); you may not use this file except in compliance\n' + '# with the License. You may obtain a copy of the License at\n' + '#\n' + '# http://www.apache.org/licenses/LICENSE-2.0\n' + '#\n' + '# Unless required by applicable law or agreed to in writing, software\n' + '# distributed under the License is distributed on an "AS IS" BASIS,\n' + '# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n' + '# See the License for the specific language governing permissions and\n' + '# limitations under the License.\n' + '################################################################################\n' + ) + tmp_file.write(original_data) + os.rename(os.path.join(dir, tmp_file_name), os.path.join(dir, file_name)) + + +def _check_grpcio_tools_version(): + version = pkg_resources.get_distribution("grpcio-tools").parsed_version + from pkg_resources import parse_version + if version < parse_version('1.3.5') or version > parse_version('1.14.2'): + raise RuntimeError( + "Version of grpcio-tools must be between 1.3.5 and 1.14.2, got %s" % version) + + if __name__ == '__main__': - generate_proto_files(force=True) + generate_proto_files() diff --git a/flink-python/pyflink/proto/flink-fn-execution.proto b/flink-python/pyflink/proto/flink-fn-execution.proto index 190a362d92..db6582d39d 100644 --- a/flink-python/pyflink/proto/flink-fn-execution.proto +++ b/flink-python/pyflink/proto/flink-fn-execution.proto @@ -16,6 +16,8 @@ * limitations under the License. */ +// NOTE: File flink_fn_execution_pb2.py is generated from this file. Please re-generate it by calling +// gen_protos.py whenever this file is changed. syntax = "proto3"; package org.apache.flink.fn_execution.v1; diff --git a/flink-python/pyflink/testing/test_case_utils.py b/flink-python/pyflink/testing/test_case_utils.py index 21d3f09e62..888824ab7c 100644 --- a/flink-python/pyflink/testing/test_case_utils.py +++ b/flink-python/pyflink/testing/test_case_utils.py @@ -27,7 +27,6 @@ from abc import abstractmethod from py4j.java_gateway import JavaObject from py4j.protocol import Py4JJavaError -from pyflink import gen_protos from pyflink.table.sources import CsvTableSource from pyflink.dataset import ExecutionEnvironment from pyflink.datastream import StreamExecutionEnvironment @@ -75,8 +74,6 @@ class PyFlinkTestCase(unittest.TestCase): def setUpClass(cls): cls.tempdir = tempfile.mkdtemp() - gen_protos.generate_proto_files() - os.environ["FLINK_TESTING"] = "1" _find_flink_home() diff --git a/flink-python/setup.py b/flink-python/setup.py index f5b26da98e..11c1c1909e 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -23,12 +23,6 @@ import sys from shutil import copytree, copy, rmtree from setuptools import setup -from setuptools.command.install import install -from setuptools.command.build_py import build_py -from setuptools.command.develop import develop -from setuptools.command.egg_info import egg_info -from setuptools.command.sdist import sdist -from setuptools.command.test import test if sys.version_info < (2, 7): print("Python versions prior to 2.7 are not supported for PyFlink.", @@ -68,24 +62,6 @@ README_FILE_TEMP_PATH = os.path.join("pyflink", "README.txt") in_flink_source = os.path.isfile("../flink-java/src/main/java/org/apache/flink/api/java/" "ExecutionEnvironment.java") - -# We must generate protos after setup_requires are installed. -def generate_protos_first(original_cmd): - try: - # pylint: disable=wrong-import-position - from pyflink import gen_protos - - class cmd(original_cmd, object): - def run(self): - gen_protos.generate_proto_files() - super(cmd, self).run() - return cmd - except ImportError: - import warnings - warnings.warn("Could not import gen_protos, skipping proto generation.") - return original_cmd - - try: if in_flink_source: @@ -208,7 +184,7 @@ run sdist. license='https://www.apache.org/licenses/LICENSE-2.0', author='Flink Developers', author_email='dev@flink.apache.org', - install_requires=['py4j==0.10.8.1', 'python-dateutil', 'apache-beam==2.15.0', + install_requires=['py4j==0.10.8.1', 'python-dateutil==2.8.0', 'apache-beam==2.15.0', 'cloudpickle==1.2.2'], tests_require=['pytest==4.4.1'], description='Apache Flink Python API', @@ -220,15 +196,7 @@ run sdist. 'Programming Language :: Python :: 2.7', 'Programming Language :: Python :: 3.5', 'Programming Language :: Python :: 3.6', - 'Programming Language :: Python :: 3.7'], - cmdclass={ - 'build_py': generate_protos_first(build_py), - 'develop': generate_protos_first(develop), - 'egg_info': generate_protos_first(egg_info), - 'sdist': generate_protos_first(sdist), - 'test': generate_protos_first(test), - 'install': generate_protos_first(install), - }, + 'Programming Language :: Python :: 3.7'] ) finally: if in_flink_source: diff --git a/pom.xml b/pom.xml index aafec9a3fa..08978f9746 100644 --- a/pom.xml +++ b/pom.xml @@ -1422,7 +1422,6 @@ under the License. flink-python/lib/** flink-python/dev/download/** flink-python/docs/_build/** - flink-python/pyflink/fn_execution/*_pb2.py -- Gitee From 592fba617655d26fbbafdde3f03d267022cc3a94 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Sun, 29 Sep 2019 14:43:16 +0800 Subject: [PATCH 159/268] [FLINK-13360][docs] Add documentation for HBase connector for Table API & SQL This closes #9799 --- docs/dev/table/connect.md | 85 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 85 insertions(+) diff --git a/docs/dev/table/connect.md b/docs/dev/table/connect.md index 5378ae971a..860e7e2aef 100644 --- a/docs/dev/table/connect.md +++ b/docs/dev/table/connect.md @@ -49,6 +49,7 @@ The following tables list all available connectors and formats. Their mutual com | Apache Kafka | 0.10 | `flink-connector-kafka-0.10` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-kafka-0.10{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-kafka-0.10{{site.scala_version_suffix}}-{{site.version}}.jar) | | Apache Kafka | 0.11 | `flink-connector-kafka-0.11` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-kafka-0.11{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-kafka-0.11{{site.scala_version_suffix}}-{{site.version}}.jar) | | Apache Kafka | 0.11+ (`universal`) | `flink-connector-kafka` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-kafka{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-kafka{{site.scala_version_suffix}}-{{site.version}}.jar) | +| HBase | 1.4.3 | `flink-hbase` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-hbase{{site.scala_version_suffix}}/{{site.version}}/flink-hbase{{site.scala_version_suffix}}-{{site.version}}.jar) | ### Formats @@ -1075,6 +1076,90 @@ CREATE TABLE MyUserTable ( {% top %} +### HBase Connector + +Source: Batch +Sink: Batch +Sink: Streaming Append Mode +Sink: Streaming Upsert Mode +Temporal Join: Sync Mode + +The HBase connector allows for reading from and writing to an HBase cluster. + +The connector can operate in [upsert mode](#update-modes) for exchanging UPSERT/DELETE messages with the external system using a [key defined by the query](./streaming/dynamic_tables.html#table-to-stream-conversion). + +For append-only queries, the connector can also operate in [append mode](#update-modes) for exchanging only INSERT messages with the external system. + +The connector can be defined as follows: + +
+
+{% highlight yaml %} +connector: + type: hbase + version: "1.4.3" # required: currently only support "1.4.3" + + table-name: "hbase_table_name" # required: HBase table name + + zookeeper: + quorum: "localhost:2181" # required: HBase Zookeeper quorum configuration + znode.parent: "/test" # optional: the root dir in Zookeeper for HBase cluster. + # The default value is "/hbase". + + write.buffer-flush: + max-size: "10mb" # optional: writing option, determines how many size in memory of buffered + # rows to insert per round trip. This can help performance on writing to JDBC + # database. The default value is "2mb". + max-rows: 1000 # optional: writing option, determines how many rows to insert per round trip. + # This can help performance on writing to JDBC database. No default value, + # i.e. the default flushing is not depends on the number of buffered rows. + interval: "2s" # optional: writing option, sets a flush interval flushing buffered requesting + # if the interval passes, in milliseconds. Default value is "0s", which means + # no asynchronous flush thread will be scheduled. +{% endhighlight %} +
+ +
+{% highlight sql %} +CREATE TABLE MyUserTable ( + hbase_rowkey_name rowkey_type, + hbase_column_family_name1 ROW<...>, + hbase_column_family_name2 ROW<...> +) WITH ( + 'connector.type' = 'hbase', -- required: specify this table type is hbase + + 'connector.version' = '1.4.3', -- required: valid connector versions are "1.4.3" + + 'connector.table-name' = 'hbase_table_name', -- required: hbase table name + + 'connector.zookeeper.quorum' = 'localhost:2181', -- required: HBase Zookeeper quorum configuration + 'connector.zookeeper.znode.parent' = '/test', -- optional: the root dir in Zookeeper for HBase cluster. + -- The default value is "/hbase". + + 'connector.write.buffer-flush.max-size' = '10mb', -- optional: writing option, determines how many size in memory of buffered + -- rows to insert per round trip. This can help performance on writing to JDBC + -- database. The default value is "2mb". + + 'connector.write.buffer-flush.max-rows' = '1000', -- optional: writing option, determines how many rows to insert per round trip. + -- This can help performance on writing to JDBC database. No default value, + -- i.e. the default flushing is not depends on the number of buffered rows. + + 'connector.write.buffer-flush.interval' = '2s', -- optional: writing option, sets a flush interval flushing buffered requesting + -- if the interval passes, in milliseconds. Default value is "0s", which means + -- no asynchronous flush thread will be scheduled. +) +{% endhighlight %} +
+
+ +**Columns:** All the column families in HBase table must be declared as `ROW` type, the field name maps to the column family name, and the nested field names map to the column qualifier names. There is no need to declare all the families and qualifiers in the schema, users can declare what's necessary. Except the `ROW` type fields, the only one field of atomic type (e.g. `STRING`, `BIGINT`) will be recognized as row key of the table. There's no constraints on the name of row key field. + +**Temporary join:** Lookup join against HBase do not use any caching; data is always queired directly through the HBase client. + +**Java/Scala/Python API:** Java/Scala/Python APIs are not supported yet. + +{% top %} + Table Formats ------------- -- Gitee From d1d7853c45773ed3ec8b7a577993b080a45f1d77 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Thu, 10 Oct 2019 12:19:33 +0800 Subject: [PATCH 160/268] [FLINK-13361][docs] Add documentation for JDBC connector for Table API & SQL This closes #9802 --- docs/dev/table/connect.md | 137 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 137 insertions(+) diff --git a/docs/dev/table/connect.md b/docs/dev/table/connect.md index 860e7e2aef..f4f73e0685 100644 --- a/docs/dev/table/connect.md +++ b/docs/dev/table/connect.md @@ -50,6 +50,7 @@ The following tables list all available connectors and formats. Their mutual com | Apache Kafka | 0.11 | `flink-connector-kafka-0.11` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-kafka-0.11{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-kafka-0.11{{site.scala_version_suffix}}-{{site.version}}.jar) | | Apache Kafka | 0.11+ (`universal`) | `flink-connector-kafka` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-kafka{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-kafka{{site.scala_version_suffix}}-{{site.version}}.jar) | | HBase | 1.4.3 | `flink-hbase` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-hbase{{site.scala_version_suffix}}/{{site.version}}/flink-hbase{{site.scala_version_suffix}}-{{site.version}}.jar) | +| JDBC | | `flink-jdbc` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-jdbc{{site.scala_version_suffix}}/{{site.version}}/flink-jdbc{{site.scala_version_suffix}}-{{site.version}}.jar) | ### Formats @@ -1160,6 +1161,142 @@ CREATE TABLE MyUserTable ( {% top %} +### JDBC Connector + +Source: Batch +Sink: Batch +Sink: Streaming Append Mode +Sink: Streaming Upsert Mode +Temporal Join: Sync Mode + +The JDBC connector allows for reading from and writing into an JDBC client. + +The connector can operate in [upsert mode](#update-modes) for exchanging UPSERT/DELETE messages with the external system using a [key defined by the query](./streaming/dynamic_tables.html#table-to-stream-conversion). + +For append-only queries, the connector can also operate in [append mode](#update-modes) for exchanging only INSERT messages with the external system. + +To use JDBC connector, need to choose an actual driver to use. Here are drivers currently supported: + +**Supported Drivers:** + +| Name | Group Id | Artifact Id | JAR | +| :-----------| :------------------| :--------------------| :----------------| +| MySQL | mysql | mysql-connector-java | [Download](http://central.maven.org/maven2/mysql/mysql-connector-java/) | +| PostgreSQL | org.postgresql | postgresql | [Download](https://jdbc.postgresql.org/download.html) | +| Derby | org.apache.derby | derby | [Download](http://db.apache.org/derby/derby_downloads.html) | + +
+ +The connector can be defined as follows: + +
+
+{% highlight yaml %} +connector: + type: jdbc + url: "jdbc:mysql://localhost:3306/flink-test" # required: JDBC DB url + table: "jdbc_table_name" # required: jdbc table name + driver: "com.mysql.jdbc.Driver" # optional: the class name of the JDBC driver to use to connect to this URL. + # If not set, it will automatically be derived from the URL. + + username: "name" # optional: jdbc user name and password + password: "password" + + read: # scan options, optional, used when reading from table + partition: # These options must all be specified if any of them is specified. In addition, partition.num must be specified. They + # describe how to partition the table when reading in parallel from multiple tasks. partition.column must be a numeric, + # date, or timestamp column from the table in question. Notice that lowerBound and upperBound are just used to decide + # the partition stride, not for filtering the rows in table. So all rows in the table will be partitioned and returned. + # This option applies only to reading. + column: "column_name" # optional, name of the column used for partitioning the input. + num: 50 # optional, the number of partitions. + lower-bound: 500 # optional, the smallest value of the first partition. + upper-bound: 1000 # optional, the largest value of the last partition. + fetch-size: 100 # optional, Gives the reader a hint as to the number of rows that should be fetched + # from the database when reading per round trip. If the value specified is zero, then + # the hint is ignored. The default value is zero. + + lookup: # lookup options, optional, used in temporary join + cache: + max-rows: 5000 # optional, max number of rows of lookup cache, over this value, the oldest rows will + # be eliminated. "cache.max-rows" and "cache.ttl" options must all be specified if any + # of them is specified. Cache is not enabled as default. + ttl: "10s" # optional, the max time to live for each rows in lookup cache, over this time, the oldest rows + # will be expired. "cache.max-rows" and "cache.ttl" options must all be specified if any of + # them is specified. Cache is not enabled as default. + max-retries: 3 # optional, max retry times if lookup database failed + + write: # sink options, optional, used when writing into table + flush: + max-rows: 5000 # optional, flush max size (includes all append, upsert and delete records), + # over this number of records, will flush data. The default value is "5000". + interval: "2s" # optional, flush interval mills, over this time, asynchronous threads will flush data. + # The default value is "0s", which means no asynchronous flush thread will be scheduled. + max-retries: 3 # optional, max retry times if writing records to database failed. +{% endhighlight %} +
+ +
+{% highlight sql %} +CREATE TABLE MyUserTable ( + ... +) WITH ( + 'connector.type' = 'jdbc', -- required: specify this table type is jdbc + + 'connector.url' = 'jdbc:mysql://localhost:3306/flink-test', -- required: JDBC DB url + + 'connector.table' = 'jdbc_table_name', -- required: jdbc table name + + 'connector.driver' = 'com.mysql.jdbc.Driver', -- optional: the class name of the JDBC driver to use to connect to this URL. + -- If not set, it will automatically be derived from the URL. + + 'connector.username' = 'name', -- optional: jdbc user name and password + 'connector.password' = 'password', + + -- scan options, optional, used when reading from table + + -- These options must all be specified if any of them is specified. In addition, partition.num must be specified. They + -- describe how to partition the table when reading in parallel from multiple tasks. partition.column must be a numeric, + -- date, or timestamp column from the table in question. Notice that lowerBound and upperBound are just used to decide + -- the partition stride, not for filtering the rows in table. So all rows in the table will be partitioned and returned. + -- This option applies only to reading. + 'connector.read.partition.column' = 'column_name', -- optional, name of the column used for partitioning the input. + 'connector.read.partition.num' = '50', -- optional, the number of partitions. + 'connector.read.partition.lower-bound' = '500', -- optional, the smallest value of the first partition. + 'connector.read.partition.upper-bound' = '1000', -- optional, the largest value of the last partition. + + 'connector.read.fetch-size' = '100', -- optional, Gives the reader a hint as to the number of rows that should be fetched + -- from the database when reading per round trip. If the value specified is zero, then + -- the hint is ignored. The default value is zero. + + -- lookup options, optional, used in temporary join + 'connector.lookup.cache.max-rows' = '5000', -- optional, max number of rows of lookup cache, over this value, the oldest rows will + -- be eliminated. "cache.max-rows" and "cache.ttl" options must all be specified if any + -- of them is specified. Cache is not enabled as default. + 'connector.lookup.cache.ttl' = '10s', -- optional, the max time to live for each rows in lookup cache, over this time, the oldest rows + -- will be expired. "cache.max-rows" and "cache.ttl" options must all be specified if any of + -- them is specified. Cache is not enabled as default. + 'connector.lookup.max-retries' = '3', -- optional, max retry times if lookup database failed + + -- sink options, optional, used when writing into table + 'connector.write.flush.max-rows' = '5000', -- optional, flush max size (includes all append, upsert and delete records), + -- over this number of records, will flush data. The default value is "5000". + 'connector.write.flush.interval' = '2s', -- optional, flush interval mills, over this time, asynchronous threads will flush data. + -- The default value is "0s", which means no asynchronous flush thread will be scheduled. + 'connector.write.max-retries' = '3' -- optional, max retry times if writing records to database failed +) +{% endhighlight %} +
+
+ +**Upsert sink:** 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`. If a JDBC table is used as upsert sink, please make sure keys of the query is one of the unique key sets or primary key of the underlying database. This can guarantee the output result is as expected. + +**Temporary Join:** JDBC connector can be used in temporal join as a lookup source. Currently, only sync lookup mode is supported. The lookup cache options (`connector.lookup.cache.max-rows` and `connector.lookup.cache.ttl`) must all be specified if any of them is specified. The lookup cache is used to improve performance of temporal join JDBC connector by querying the cache first instead of send all requests to remote database. But the returned value might not be the latest if it is from the cache. So it's a balance between throughput and correctness. + +**Writing:** As default, the `connector.write.flush.interval` is `0s` and `connector.write.flush.max-rows` is `5000`, which means for low traffic queries, the buffered output rows may not be flushed to database for a long time. So the interval configuration is recommended to set. + +{% top %} + Table Formats ------------- -- Gitee From 17fae0e0273c031118bdedbcdb8a03c401257690 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 9 Oct 2019 15:10:09 +0200 Subject: [PATCH 161/268] [FLINK-14353][table] Enable fork reuse for tests --- flink-table/flink-table-planner/pom.xml | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/flink-table/flink-table-planner/pom.xml b/flink-table/flink-table-planner/pom.xml index c8fc42f653..bb92960053 100644 --- a/flink-table/flink-table-planner/pom.xml +++ b/flink-table/flink-table-planner/pom.xml @@ -406,6 +406,27 @@ under the License. ${project.basedir}/../../tools/maven/scalastyle-config.xml + + + org.apache.maven.plugins + maven-surefire-plugin + + + integration-tests + integration-test + + test + + + + **/*ITCase.* + + + true + + + + -- Gitee From d231032595cfb292d2ed1c42725493e765210fc5 Mon Sep 17 00:00:00 2001 From: caoyingjie Date: Thu, 10 Oct 2019 16:11:45 +0800 Subject: [PATCH 162/268] [FLINK-14118][runtime] Reduce the unnecessary flushing when there is no data available for flush. (#9706) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The purpose of this commit is to reduce unnecessary flushing when there is no data available for flush. More specifically, when there is exactly one BufferConsumer in the buffer queue of subpartition and no new data will be added for a while in the future (may because of just no input or the logic of the operator is to collect some data for processing and will not emit records immediately), the previous implementation will continuously notify data available (may wake up the netty thread), which is unnecessary, and eliminating these unnecessary flush can reduce CPU usage by 20% - 40% for some jobs. Benchmark results before the fix: Benchmark Mode Cnt Score Error Units networkSkewedThroughput thrpt 30 18240.197 ± 1892.419 ops/ms After the fix: networkSkewedThroughput thrpt 30 24532.313 ± 1118.312 ops/m --- .../runtime/io/network/buffer/BufferConsumer.java | 11 +++++++++++ .../io/network/partition/PipelinedSubpartition.java | 4 ++-- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java index c06c233746..ecebc67344 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferConsumer.java @@ -141,6 +141,13 @@ public class BufferConsumer implements Closeable { return currentReaderPosition; } + /** + * Returns true if there is new data available for reading. + */ + public boolean isDataAvailable() { + return currentReaderPosition < writerPosition.getLatest(); + } + /** * Cached reading wrapper around {@link PositionMarker}. * @@ -168,6 +175,10 @@ public class BufferConsumer implements Closeable { return PositionMarker.getAbsolute(cachedPosition); } + private int getLatest() { + return PositionMarker.getAbsolute(positionMarker.get()); + } + private void update() { this.cachedPosition = positionMarker.get(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java index 0c38ae2c2c..ecf695696a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java @@ -307,8 +307,8 @@ class PipelinedSubpartition extends ResultSubpartition { } // if there is more then 1 buffer, we already notified the reader // (at the latest when adding the second buffer) - notifyDataAvailable = !flushRequested && buffers.size() == 1; - flushRequested = true; + notifyDataAvailable = !flushRequested && buffers.size() == 1 && buffers.peek().isDataAvailable(); + flushRequested = flushRequested || buffers.size() > 1 || notifyDataAvailable; } if (notifyDataAvailable) { notifyDataAvailable(); -- Gitee From a5198b891da02434ceb1af17a12abf46b1e7e377 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Mon, 7 Oct 2019 10:40:58 +0200 Subject: [PATCH 163/268] [FLINK-14334][es][docs] Use ExceptionUtils#findThrowable This closes #9849. --- docs/dev/connectors/elasticsearch.md | 8 ++++---- docs/dev/connectors/elasticsearch.zh.md | 8 ++++---- .../connectors/cassandra/CassandraFailureHandler.java | 2 +- .../elasticsearch/ActionRequestFailureHandler.java | 4 ++-- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md index 45b041959a..597b89b921 100644 --- a/docs/dev/connectors/elasticsearch.md +++ b/docs/dev/connectors/elasticsearch.md @@ -366,10 +366,10 @@ input.addSink(new ElasticsearchSink<>( int restStatusCode, RequestIndexer indexer) throw Throwable { - if (ExceptionUtils.containsThrowable(failure, EsRejectedExecutionException.class)) { + if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) { // full queue; re-add document for indexing indexer.add(action); - } else if (ExceptionUtils.containsThrowable(failure, ElasticsearchParseException.class)) { + } else if (ExceptionUtils.findThrowable(failure, ElasticsearchParseException.class).isPresent()) { // malformed document; simply drop request without failing sink } else { // for all other failures, fail the sink @@ -394,10 +394,10 @@ input.addSink(new ElasticsearchSink( int restStatusCode, RequestIndexer indexer) { - if (ExceptionUtils.containsThrowable(failure, EsRejectedExecutionException.class)) { + if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) { // full queue; re-add document for indexing indexer.add(action) - } else if (ExceptionUtils.containsThrowable(failure, ElasticsearchParseException.class)) { + } else if (ExceptionUtils.findThrowable(failure, ElasticsearchParseException.class).isPresent()) { // malformed document; simply drop request without failing sink } else { // for all other failures, fail the sink diff --git a/docs/dev/connectors/elasticsearch.zh.md b/docs/dev/connectors/elasticsearch.zh.md index aed915f7d5..1fba302c30 100644 --- a/docs/dev/connectors/elasticsearch.zh.md +++ b/docs/dev/connectors/elasticsearch.zh.md @@ -366,10 +366,10 @@ input.addSink(new ElasticsearchSink<>( int restStatusCode, RequestIndexer indexer) throw Throwable { - if (ExceptionUtils.containsThrowable(failure, EsRejectedExecutionException.class)) { + if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) { // full queue; re-add document for indexing indexer.add(action); - } else if (ExceptionUtils.containsThrowable(failure, ElasticsearchParseException.class)) { + } else if (ExceptionUtils.findThrowable(failure, ElasticsearchParseException.class).isPresent()) { // malformed document; simply drop request without failing sink } else { // for all other failures, fail the sink @@ -394,10 +394,10 @@ input.addSink(new ElasticsearchSink( int restStatusCode, RequestIndexer indexer) { - if (ExceptionUtils.containsThrowable(failure, EsRejectedExecutionException.class)) { + if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) { // full queue; re-add document for indexing indexer.add(action) - } else if (ExceptionUtils.containsThrowable(failure, ElasticsearchParseException.class)) { + } else if (ExceptionUtils.findThrowable(failure, ElasticsearchParseException.class).isPresent()) { // malformed document; simply drop request without failing sink } else { // for all other failures, fail the sink diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraFailureHandler.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraFailureHandler.java index 478ba47129..8bcb0ca5ff 100644 --- a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraFailureHandler.java +++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraFailureHandler.java @@ -34,7 +34,7 @@ import java.io.Serializable; * * @Override * void onFailure(Throwable failure) throws IOException { - * if (ExceptionUtils.containsThrowable(failure, WriteTimeoutException.class)) { + * if (ExceptionUtils.findThrowable(failure, WriteTimeoutException.class).isPresent()) { * // drop exception * } else { * // for all other failures, fail the sink; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java index 260f80e586..1514aba61a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java @@ -36,10 +36,10 @@ import java.io.Serializable; * * @Override * void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { - * if (ExceptionUtils.containsThrowable(failure, EsRejectedExecutionException.class)) { + * if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) { * // full queue; re-add document for indexing * indexer.add(action); - * } else if (ExceptionUtils.containsThrowable(failure, ElasticsearchParseException.class)) { + * } else if (ExceptionUtils.findThrowable(failure, ElasticsearchParseException.class).isPresent()) { * // malformed document; simply drop request without failing sink * } else { * // for all other failures, fail the sink; -- Gitee From 11c26109dd647baf59c1c9da9934c46c8d9f1ff7 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 7 Jun 2019 21:31:12 +0200 Subject: [PATCH 164/268] [hotfix] Remove Scala Promise from LeaderRetrievalUtils --- .../runtime/util/LeaderRetrievalUtils.java | 23 +++++++------------ 1 file changed, 8 insertions(+), 15 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java index 3012bbb102..ad746a52b5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderRetrievalUtils.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.util; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -30,11 +29,9 @@ import org.slf4j.LoggerFactory; import java.net.InetAddress; import java.time.Duration; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import scala.concurrent.Future; -import scala.concurrent.Promise; - /** * Utility class to work with {@link LeaderRetrievalService} class. */ @@ -62,9 +59,7 @@ public class LeaderRetrievalUtils { try { leaderRetrievalService.start(listener); - Future connectionInfoFuture = listener.getLeaderConnectionInfoFuture(); - - return FutureUtils.toJava(connectionInfoFuture).get(timeout.toMillis(), TimeUnit.MILLISECONDS); + return listener.getLeaderConnectionInfoFuture().get(timeout.toMillis(), TimeUnit.MILLISECONDS); } catch (Exception e) { throw new LeaderRetrievalException("Could not retrieve the leader address and leader " + "session ID.", e); @@ -110,25 +105,23 @@ public class LeaderRetrievalUtils { * leader's akka URL and the current leader session ID. */ public static class LeaderConnectionInfoListener implements LeaderRetrievalListener { - private final Promise connectionInfo = new scala.concurrent.impl.Promise.DefaultPromise<>(); + private final CompletableFuture connectionInfoFuture = new CompletableFuture<>(); - public Future getLeaderConnectionInfoFuture() { - return connectionInfo.future(); + public CompletableFuture getLeaderConnectionInfoFuture() { + return connectionInfoFuture; } @Override public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) { - if (leaderAddress != null && !leaderAddress.equals("") && !connectionInfo.isCompleted()) { + if (leaderAddress != null && !leaderAddress.equals("") && !connectionInfoFuture.isDone()) { final LeaderConnectionInfo leaderConnectionInfo = new LeaderConnectionInfo(leaderSessionID, leaderAddress); - connectionInfo.success(leaderConnectionInfo); + connectionInfoFuture.complete(leaderConnectionInfo); } } @Override public void handleError(Exception exception) { - if (!connectionInfo.isCompleted()) { - connectionInfo.failure(exception); - } + connectionInfoFuture.completeExceptionally(exception); } } -- Gitee From 7e7ee2812f459bcfdab8862db8f6db2cab0e1368 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 7 Jun 2019 16:07:21 +0200 Subject: [PATCH 165/268] [FLINK-14287] Decouple leader address from LeaderContender Change LeaderElectionService#confirmLeadership to accept leader address so that the LeaderContender does not need to know the address of the potential leader before gaining leadership. This allows to decouple the leader election from the actual leader component. This closes #9813. --- .../flink/runtime/dispatcher/Dispatcher.java | 2 +- .../nonha/embedded/EmbeddedLeaderService.java | 17 +++--- .../SingleLeaderElectionService.java | 7 +-- .../jobmaster/JobManagerRunnerImpl.java | 13 +++- .../leaderelection/LeaderElectionService.java | 20 ++++--- .../StandaloneLeaderElectionService.java | 2 +- .../ZooKeeperLeaderElectionService.java | 59 +++++++++++-------- .../resourcemanager/ResourceManager.java | 2 +- .../webmonitor/WebMonitorEndpoint.java | 2 +- .../embedded/EmbeddedHaServicesTest.java | 8 +-- .../SingleLeaderElectionServiceTest.java | 2 +- .../leaderelection/LeaderElectionTest.java | 2 +- .../leaderelection/TestingContender.java | 2 +- .../TestingLeaderElectionService.java | 2 +- .../ResourceManagerHATest.java | 2 +- .../DocumentingDispatcherRestEndpoint.java | 2 +- .../YarnIntraNonHaMasterServicesTest.java | 2 +- 17 files changed, 85 insertions(+), 61 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index bf4518cd89..1c7fb6a414 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -896,7 +896,7 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme recoveredJobsFuture, BiFunctionWithException.unchecked((Boolean confirmLeadership, Collection recoveredJobs) -> { if (confirmLeadership) { - leaderElectionService.confirmLeaderSessionID(newLeaderSessionID); + leaderElectionService.confirmLeadership(newLeaderSessionID, getAddress()); } else { for (JobGraph recoveredJob : recoveredJobs) { jobGraphStore.releaseJobGraph(recoveredJob.getJobID()); 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 e3e98e1090..0cf37ddf9f 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 @@ -236,7 +236,10 @@ public class EmbeddedLeaderService { /** * Callback from leader contenders when they confirm a leader grant. */ - private void confirmLeader(final EmbeddedLeaderElectionService service, final UUID leaderSessionId) { + private void confirmLeader( + final EmbeddedLeaderElectionService service, + final UUID leaderSessionId, + final String leaderAddress) { synchronized (lock) { // if the service was shut down in the meantime, ignore this confirmation if (!service.running || shutdown) { @@ -246,16 +249,15 @@ public class EmbeddedLeaderService { try { // check if the confirmation is for the same grant, or whether it is a stale grant if (service == currentLeaderProposed && currentLeaderSessionId.equals(leaderSessionId)) { - final String address = service.contender.getAddress(); - LOG.info("Received confirmation of leadership for leader {} , session={}", address, leaderSessionId); + LOG.info("Received confirmation of leadership for leader {} , session={}", leaderAddress, leaderSessionId); // mark leadership currentLeaderConfirmed = service; - currentLeaderAddress = address; + currentLeaderAddress = leaderAddress; currentLeaderProposed = null; // notify all listeners - notifyAllListeners(address, leaderSessionId); + notifyAllListeners(leaderAddress, leaderSessionId); } else { LOG.debug("Received confirmation of leadership for a stale leadership grant. Ignoring."); @@ -434,9 +436,10 @@ public class EmbeddedLeaderService { } @Override - public void confirmLeaderSessionID(UUID leaderSessionID) { + public void confirmLeadership(UUID leaderSessionID, String leaderAddress) { checkNotNull(leaderSessionID); - confirmLeader(this, leaderSessionID); + checkNotNull(leaderAddress); + confirmLeader(this, leaderSessionID, leaderAddress); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionService.java index 89b297722e..879b52356c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionService.java @@ -141,7 +141,7 @@ public class SingleLeaderElectionService implements LeaderElectionService { } @Override - public void confirmLeaderSessionID(UUID leaderSessionID) { + public void confirmLeadership(UUID leaderSessionID, String leaderAddress) { checkNotNull(leaderSessionID, "leaderSessionID"); checkArgument(leaderSessionID.equals(leaderId), "confirmed wrong leader session id"); @@ -151,14 +151,13 @@ public class SingleLeaderElectionService implements LeaderElectionService { checkState(leader == null, "leader already confirmed"); // accept the confirmation - final String address = proposedLeader.getAddress(); - leaderAddress = address; + this.leaderAddress = leaderAddress; leader = proposedLeader; // notify all listeners for (EmbeddedLeaderRetrievalService listener : listeners) { notificationExecutor.execute( - new NotifyOfLeaderCall(address, leaderId, listener.listener, LOG)); + new NotifyOfLeaderCall(leaderAddress, leaderId, listener.listener, LOG)); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java index 7b2609a65d..52740a64f6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java @@ -335,7 +335,10 @@ public class JobManagerRunnerImpl implements LeaderContender, OnCompletionAction final CompletableFuture currentLeaderGatewayFuture = leaderGatewayFuture; return startFuture.thenAcceptAsync( - (Acknowledge ack) -> confirmLeaderSessionIdIfStillLeader(leaderSessionId, currentLeaderGatewayFuture), + (Acknowledge ack) -> confirmLeaderSessionIdIfStillLeader( + leaderSessionId, + jobMasterService.getAddress(), + currentLeaderGatewayFuture), executor); } @@ -358,10 +361,14 @@ public class JobManagerRunnerImpl implements LeaderContender, OnCompletionAction } } - private void confirmLeaderSessionIdIfStillLeader(UUID leaderSessionId, CompletableFuture currentLeaderGatewayFuture) { + private void confirmLeaderSessionIdIfStillLeader( + UUID leaderSessionId, + String leaderAddress, + CompletableFuture currentLeaderGatewayFuture) { + if (leaderElectionService.hasLeadership(leaderSessionId)) { currentLeaderGatewayFuture.complete(jobMasterService.getGateway()); - leaderElectionService.confirmLeaderSessionID(leaderSessionId); + leaderElectionService.confirmLeadership(leaderSessionId, leaderAddress); } else { log.debug("Ignoring confirmation of leader session id because {} is no longer the leader.", getAddress()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java index 10f2f68efa..c7a5588009 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java @@ -30,9 +30,9 @@ import java.util.UUID; * to instantiate its own leader election service. * * Once a contender has been granted leadership he has to confirm the received leader session ID - * by calling the method confirmLeaderSessionID. This will notify the leader election service, that - * the contender has received the new leader session ID and that it can now be published for - * leader retrieval services. + * by calling the method {@link #confirmLeadership(UUID, String)}. This will notify the leader election + * service, that the contender has accepted the leadership specified and that the leader session id as + * well as the leader address can now be published for leader retrieval services. */ public interface LeaderElectionService { @@ -51,16 +51,18 @@ public interface LeaderElectionService { void stop() throws Exception; /** - * Confirms that the new leader session ID has been successfully received by the new leader. - * This method is usually called by the newly appointed {@link LeaderContender}. + * Confirms that the {@link LeaderContender} has accepted the leadership identified by the + * given leader session id. It also publishes the leader address under which the leader is + * reachable. * - * The rational behind this method is to establish an order between setting the new leader - * session ID in the {@link LeaderContender} and publishing the new leader session ID to the - * leader retrieval services. + *

The rational behind this method is to establish an order between setting the new leader + * session ID in the {@link LeaderContender} and publishing the new leader session ID as well + * as the leader address to the leader retrieval services. * * @param leaderSessionID The new leader session ID + * @param leaderAddress The address of the new leader */ - void confirmLeaderSessionID(UUID leaderSessionID); + void confirmLeadership(UUID leaderSessionID, String leaderAddress); /** * Returns true if the {@link LeaderContender} with which the service has been started owns diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionService.java index ec997a332b..d0bb072898 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionService.java @@ -57,7 +57,7 @@ public class StandaloneLeaderElectionService implements LeaderElectionService { } @Override - public void confirmLeaderSessionID(UUID leaderSessionID) {} + public void confirmLeadership(UUID leaderSessionID, String leaderAddress) {} @Override public boolean hasLeadership(@Nonnull UUID leaderSessionId) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java index 87684c871d..91227dca63 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java @@ -72,6 +72,8 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le private volatile UUID confirmedLeaderSessionID; + private volatile String confirmedLeaderAddress; + /** The leader contender which applies for leadership. */ private volatile LeaderContender leaderContender; @@ -100,6 +102,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le issuedLeaderSessionID = null; confirmedLeaderSessionID = null; + confirmedLeaderAddress = null; leaderContender = null; running = false; @@ -177,12 +180,12 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le } @Override - public void confirmLeaderSessionID(UUID leaderSessionID) { + public void confirmLeadership(UUID leaderSessionID, String leaderAddress) { if (LOG.isDebugEnabled()) { LOG.debug( "Confirm leader session ID {} for leader {}.", leaderSessionID, - leaderContender.getAddress()); + leaderAddress); } Preconditions.checkNotNull(leaderSessionID); @@ -192,8 +195,8 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le synchronized (lock) { if (running) { if (leaderSessionID.equals(this.issuedLeaderSessionID)) { - confirmedLeaderSessionID = leaderSessionID; - writeLeaderInformation(confirmedLeaderSessionID); + confirmLeaderInformation(leaderSessionID, leaderAddress); + writeLeaderInformation(); } } else { LOG.debug("Ignoring the leader session Id {} confirmation, since the " + @@ -206,6 +209,11 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le } } + private void confirmLeaderInformation(UUID leaderSessionID, String leaderAddress) { + confirmedLeaderSessionID = leaderSessionID; + confirmedLeaderAddress = leaderAddress; + } + @Override public boolean hasLeadership(@Nonnull UUID leaderSessionId) { return leaderLatch.hasLeadership() && leaderSessionId.equals(issuedLeaderSessionID); @@ -216,7 +224,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le synchronized (lock) { if (running) { issuedLeaderSessionID = UUID.randomUUID(); - confirmedLeaderSessionID = null; + clearConfirmedLeaderInformation(); if (LOG.isDebugEnabled()) { LOG.debug( @@ -233,16 +241,23 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le } } + private void clearConfirmedLeaderInformation() { + confirmedLeaderSessionID = null; + confirmedLeaderAddress = null; + } + @Override public void notLeader() { synchronized (lock) { if (running) { - issuedLeaderSessionID = null; - confirmedLeaderSessionID = null; + LOG.debug( + "Revoke leadership of {} ({}@{}).", + leaderContender, + confirmedLeaderSessionID, + confirmedLeaderAddress); - if (LOG.isDebugEnabled()) { - LOG.debug("Revoke leadership of {}.", leaderContender.getAddress()); - } + issuedLeaderSessionID = null; + clearConfirmedLeaderInformation(); leaderContender.revokeLeadership(); } else { @@ -275,7 +290,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le "Writing leader information into empty node by {}.", leaderContender.getAddress()); } - writeLeaderInformation(confirmedLeaderSessionID); + writeLeaderInformation(); } else { byte[] data = childData.getData(); @@ -286,7 +301,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le "Writing leader information into node with empty data field by {}.", leaderContender.getAddress()); } - writeLeaderInformation(confirmedLeaderSessionID); + writeLeaderInformation(); } else { ByteArrayInputStream bais = new ByteArrayInputStream(data); ObjectInputStream ois = new ObjectInputStream(bais); @@ -294,7 +309,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le String leaderAddress = ois.readUTF(); UUID leaderSessionID = (UUID) ois.readObject(); - if (!leaderAddress.equals(this.leaderContender.getAddress()) || + if (!leaderAddress.equals(confirmedLeaderAddress) || (leaderSessionID == null || !leaderSessionID.equals(confirmedLeaderSessionID))) { // the data field does not correspond to the expected leader information if (LOG.isDebugEnabled()) { @@ -302,7 +317,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le "Correcting leader information by {}.", leaderContender.getAddress()); } - writeLeaderInformation(confirmedLeaderSessionID); + writeLeaderInformation(); } } } @@ -320,24 +335,22 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le /** * Writes the current leader's address as well the given leader session ID to ZooKeeper. - * - * @param leaderSessionID Leader session ID which is written to ZooKeeper */ - protected void writeLeaderInformation(UUID leaderSessionID) { + protected void writeLeaderInformation() { // this method does not have to be synchronized because the curator framework client // is thread-safe try { if (LOG.isDebugEnabled()) { LOG.debug( "Write leader information: Leader={}, session ID={}.", - leaderContender.getAddress(), - leaderSessionID); + confirmedLeaderAddress, + confirmedLeaderSessionID); } ByteArrayOutputStream baos = new ByteArrayOutputStream(); ObjectOutputStream oos = new ObjectOutputStream(baos); - oos.writeUTF(leaderContender.getAddress()); - oos.writeObject(leaderSessionID); + oos.writeUTF(confirmedLeaderAddress); + oos.writeObject(confirmedLeaderSessionID); oos.close(); @@ -381,8 +394,8 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le if (LOG.isDebugEnabled()) { LOG.debug( "Successfully wrote leader information: Leader={}, session ID={}.", - leaderContender.getAddress(), - leaderSessionID); + confirmedLeaderAddress, + confirmedLeaderSessionID); } } catch (Exception e) { leaderContender.handleError( 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 8698e842ac..56450ff1c8 100755 --- 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 @@ -901,7 +901,7 @@ public abstract class ResourceManager (acceptLeadership) -> { if (acceptLeadership) { // confirming the leader session ID might be blocking, - leaderElectionService.confirmLeaderSessionID(newLeaderSessionID); + leaderElectionService.confirmLeadership(newLeaderSessionID, getAddress()); } }, getRpcService().getExecutor()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index 1c31edb1ef..f474ad50df 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -710,7 +710,7 @@ public class WebMonitorEndpoint extends RestServerEndp @Override public void grantLeadership(final UUID leaderSessionID) { log.info("{} was granted leadership with leaderSessionID={}", getRestBaseUrl(), leaderSessionID); - leaderElectionService.confirmLeaderSessionID(leaderSessionID); + leaderElectionService.confirmLeadership(leaderSessionID, getRestBaseUrl()); } @Override 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 7f06d60c94..3b9ce6513b 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 @@ -137,7 +137,7 @@ public class EmbeddedHaServicesTest extends TestLogger { final UUID leaderId = leaderIdArgumentCaptor.getValue(); - leaderElectionService.confirmLeaderSessionID(leaderId); + leaderElectionService.confirmLeadership(leaderId, address); verify(leaderRetrievalListener).notifyLeaderAddress(eq(address), eq(leaderId)); } @@ -163,7 +163,7 @@ public class EmbeddedHaServicesTest extends TestLogger { final UUID leaderId = leaderIdArgumentCaptor.getValue(); - leaderElectionService.confirmLeaderSessionID(leaderId); + leaderElectionService.confirmLeadership(leaderId, address); verify(leaderRetrievalListener).notifyLeaderAddress(eq(address), eq(leaderId)); } @@ -191,8 +191,8 @@ public class EmbeddedHaServicesTest extends TestLogger { assertThat(dispatcherLeaderElectionService.hasLeadership(newLeaderSessionId), is(true)); - dispatcherLeaderElectionService.confirmLeaderSessionID(oldLeaderSessionId); - dispatcherLeaderElectionService.confirmLeaderSessionID(newLeaderSessionId); + dispatcherLeaderElectionService.confirmLeadership(oldLeaderSessionId, leaderContender.getAddress()); + dispatcherLeaderElectionService.confirmLeadership(newLeaderSessionId, leaderContender.getAddress()); assertThat(dispatcherLeaderElectionService.hasLeadership(newLeaderSessionId), is(true)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java index 3875c4f171..535679ea65 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java @@ -216,7 +216,7 @@ public class SingleLeaderElectionServiceTest { @Override public Void answer(InvocationOnMock invocation) throws Throwable { final UUID uuid = (UUID) invocation.getArguments()[0]; - service.confirmLeaderSessionID(uuid); + service.confirmLeadership(uuid, address); return null; } }).when(mockContender).grantLeadership(any(UUID.class)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionTest.java index 44016f82cc..9090cae6c2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionTest.java @@ -101,7 +101,7 @@ public class LeaderElectionTest extends TestLogger { assertThat(leaderElectionService.hasLeadership(leaderSessionId), is(true)); assertThat(leaderElectionService.hasLeadership(UUID.randomUUID()), is(false)); - leaderElectionService.confirmLeaderSessionID(leaderSessionId); + leaderElectionService.confirmLeadership(leaderSessionId, manualLeaderContender.getAddress()); assertThat(leaderElectionService.hasLeadership(leaderSessionId), is(true)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingContender.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingContender.java index b03c1654c7..34408ff2ea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingContender.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingContender.java @@ -118,7 +118,7 @@ public class TestingContender implements LeaderContender { this.leaderSessionID = leaderSessionID; - leaderElectionService.confirmLeaderSessionID(leaderSessionID); + leaderElectionService.confirmLeadership(leaderSessionID, getAddress()); leader = true; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java index e60f2f776a..f282ebdc4d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java @@ -67,7 +67,7 @@ public class TestingLeaderElectionService implements LeaderElectionService { } @Override - public synchronized void confirmLeaderSessionID(UUID leaderSessionID) { + public synchronized void confirmLeadership(UUID leaderSessionID, String leaderAddress) { if (confirmationFuture != null) { confirmationFuture.complete(leaderSessionID); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java index 08f8a0f045..ff2f1ebc30 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java @@ -52,7 +52,7 @@ public class ResourceManagerHATest extends TestLogger { TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService() { @Override - public void confirmLeaderSessionID(UUID leaderId) { + public void confirmLeadership(UUID leaderId, String leaderAddress) { leaderSessionIdFuture.complete(leaderId); } }; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/util/DocumentingDispatcherRestEndpoint.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/util/DocumentingDispatcherRestEndpoint.java index 96e17e95e7..db45ee02cb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/util/DocumentingDispatcherRestEndpoint.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/util/DocumentingDispatcherRestEndpoint.java @@ -110,7 +110,7 @@ public class DocumentingDispatcherRestEndpoint extends DispatcherRestEndpoint im } @Override - public void confirmLeaderSessionID(final UUID leaderSessionID) { + public void confirmLeadership(final UUID leaderSessionID, final String leaderAddress) { } diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java index bdc7863c79..0d09f87902 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java @@ -156,7 +156,7 @@ public class YarnIntraNonHaMasterServicesTest extends TestLogger { @Override public Void answer(InvocationOnMock invocation) throws Throwable { final UUID uuid = (UUID) invocation.getArguments()[0]; - service.confirmLeaderSessionID(uuid); + service.confirmLeadership(uuid, address); return null; } }).when(mockContender).grantLeadership(any(UUID.class)); -- Gitee From d963eabdbc9ee59f4606c4f9bb6465ee1e88f0e5 Mon Sep 17 00:00:00 2001 From: "Wind (Jiayi Liao)" Date: Fri, 11 Oct 2019 04:07:22 +0800 Subject: [PATCH 166/268] [FLINK-14309] [test-stability] Add retries and acks config in producer test (#9835) Add retries and acks config to reduce the risk of failure of flush() method. --- .../streaming/connectors/kafka/KafkaProducerTestBase.java | 3 +++ 1 file changed, 3 insertions(+) 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 9616e9138b..a971386c44 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 @@ -247,6 +247,9 @@ public abstract class KafkaProducerTestBase extends KafkaTestBaseWithFlink { // increase batch.size and linger.ms - this tells KafkaProducer to batch produced events instead of flushing them immediately properties.setProperty("batch.size", "10240000"); properties.setProperty("linger.ms", "10000"); + // kafka producer messages guarantee + properties.setProperty("retries", "3"); + properties.setProperty("acks", "all"); BrokerRestartingMapper.resetState(kafkaServer::blockProxyTraffic); -- Gitee From 2750b2ea29c440c6e699f756e5cd3431653fc49f Mon Sep 17 00:00:00 2001 From: Liupengcheng Date: Tue, 8 Oct 2019 14:09:58 +0800 Subject: [PATCH 167/268] [FLINK-14341][python] Support pip versions of 7.1.x for gen_protos.py Previously, only pip versions of >=8.0.0 are supported for gen_protos.py, this commit also add support for pip versions of 7.1.x. This closes #9851. --- flink-python/README.md | 2 +- flink-python/pyflink/gen_protos.py | 18 ++++++++++++++---- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/flink-python/README.md b/flink-python/README.md index 76df93dd77..29dc3c419f 100644 --- a/flink-python/README.md +++ b/flink-python/README.md @@ -40,4 +40,4 @@ python pyflink/gen_protos.py PyFlink depends on the following libraries to execute the above script: 1. grpcio-tools (>=1.3.5,<=1.14.2) 2. setuptools (>=37.0.0) -3. pip (>=8.0.0) +3. pip (>=7.1.0) diff --git a/flink-python/pyflink/gen_protos.py b/flink-python/pyflink/gen_protos.py index de54d5fb05..b5908029a0 100644 --- a/flink-python/pyflink/gen_protos.py +++ b/flink-python/pyflink/gen_protos.py @@ -120,10 +120,20 @@ def _install_grpcio_tools_and_generate_proto_files(force, output_dir): logging.warning('Installing grpcio-tools into %s', install_path) try: start = time.time() - subprocess.check_call( - [sys.executable, '-m', 'pip', 'install', - '--prefix', install_path, '--build', build_path, - '--upgrade', GRPC_TOOLS, "-I"]) + # since '--prefix' option is only supported for pip 8.0+, so here we fallback to + # use '--install-option' when the pip version is lower than 8.0.0. + pip_version = pkg_resources.get_distribution("pip").version + from pkg_resources import parse_version + if parse_version(pip_version) >= parse_version('8.0.0'): + subprocess.check_call( + [sys.executable, '-m', 'pip', 'install', + '--prefix', install_path, '--build', build_path, + '--upgrade', GRPC_TOOLS, "-I"]) + else: + subprocess.check_call( + [sys.executable, '-m', 'pip', 'install', + '--install-option', '--prefix=' + install_path, '--build', build_path, + '--upgrade', GRPC_TOOLS, "-I"]) from distutils.dist import Distribution install_obj = Distribution().get_command_obj('install', create=True) install_obj.prefix = install_path -- Gitee From 71e440d5f62fbd43df18f999b7818de48a279eb0 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Mon, 30 Sep 2019 11:15:53 +0800 Subject: [PATCH 168/268] [FLINK-14289][runtime] Remove Optional fields from RecordWriter relevant classes Based on the code style guides for Jave Optional, it should not be used for class fields. So we remove the optional usages from RecordWriter, BroadcastRecordWriter and ChannelSelectorRecordWriter. --- .../api/writer/BroadcastRecordWriter.java | 34 +++++++++---------- .../writer/ChannelSelectorRecordWriter.java | 31 ++++++++--------- .../io/network/api/writer/RecordWriter.java | 18 +++++----- 3 files changed, 40 insertions(+), 43 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java index 7544af2017..b4999835d7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java @@ -23,8 +23,9 @@ import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; +import javax.annotation.Nullable; + import java.io.IOException; -import java.util.Optional; import static org.apache.flink.util.Preconditions.checkState; @@ -40,7 +41,8 @@ import static org.apache.flink.util.Preconditions.checkState; public final class BroadcastRecordWriter extends RecordWriter { /** The current buffer builder shared for all the channels. */ - private Optional bufferBuilder = Optional.empty(); + @Nullable + private BufferBuilder bufferBuilder; /** * The flag for judging whether {@link #requestNewBufferBuilder(int)} and {@link #flushTargetPartition(int)} @@ -79,10 +81,10 @@ public final class BroadcastRecordWriter extends R emit(record, targetChannelIndex); randomTriggered = false; - if (bufferBuilder.isPresent()) { + if (bufferBuilder != null) { for (int index = 0; index < numberOfChannels; index++) { if (index != targetChannelIndex) { - targetPartition.addBufferConsumer(bufferBuilder.get().createBufferConsumer(), index); + targetPartition.addBufferConsumer(bufferBuilder.createBufferConsumer(), index); } } } @@ -111,11 +113,7 @@ public final class BroadcastRecordWriter extends R @Override public BufferBuilder getBufferBuilder(int targetChannel) throws IOException, InterruptedException { - if (bufferBuilder.isPresent()) { - return bufferBuilder.get(); - } else { - return requestNewBufferBuilder(targetChannel); - } + return bufferBuilder != null ? bufferBuilder : requestNewBufferBuilder(targetChannel); } /** @@ -128,7 +126,7 @@ public final class BroadcastRecordWriter extends R */ @Override public BufferBuilder requestNewBufferBuilder(int targetChannel) throws IOException, InterruptedException { - checkState(!bufferBuilder.isPresent() || bufferBuilder.get().isFinished()); + checkState(bufferBuilder == null || bufferBuilder.isFinished()); BufferBuilder builder = targetPartition.getBufferBuilder(); if (randomTriggered) { @@ -141,25 +139,25 @@ public final class BroadcastRecordWriter extends R } } - bufferBuilder = Optional.of(builder); + bufferBuilder = builder; return builder; } @Override public void tryFinishCurrentBufferBuilder(int targetChannel) { - if (!bufferBuilder.isPresent()) { + if (bufferBuilder == null) { return; } - BufferBuilder builder = bufferBuilder.get(); - bufferBuilder = Optional.empty(); + BufferBuilder builder = bufferBuilder; + bufferBuilder = null; finishBufferBuilder(builder); } @Override public void emptyCurrentBufferBuilder(int targetChannel) { - bufferBuilder = Optional.empty(); + bufferBuilder = null; } @Override @@ -173,9 +171,9 @@ public final class BroadcastRecordWriter extends R } private void closeBufferBuilder() { - if (bufferBuilder.isPresent()) { - bufferBuilder.get().finish(); - bufferBuilder = Optional.empty(); + if (bufferBuilder != null) { + bufferBuilder.finish(); + bufferBuilder = null; } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java index eaeb5091f6..5f5e5964b9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java @@ -22,7 +22,6 @@ import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import java.io.IOException; -import java.util.Optional; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -40,7 +39,8 @@ public final class ChannelSelectorRecordWriter ext private final ChannelSelector channelSelector; - private final Optional[] bufferBuilders; + /** Every subpartition maintains a separate buffer builder which might be null. */ + private final BufferBuilder[] bufferBuilders; ChannelSelectorRecordWriter( ResultPartitionWriter writer, @@ -52,10 +52,7 @@ public final class ChannelSelectorRecordWriter ext this.channelSelector = checkNotNull(channelSelector); this.channelSelector.setup(numberOfChannels); - this.bufferBuilders = new Optional[numberOfChannels]; - for (int i = 0; i < numberOfChannels; i++) { - bufferBuilders[i] = Optional.empty(); - } + this.bufferBuilders = new BufferBuilder[numberOfChannels]; } @Override @@ -92,8 +89,8 @@ public final class ChannelSelectorRecordWriter ext @Override public BufferBuilder getBufferBuilder(int targetChannel) throws IOException, InterruptedException { - if (bufferBuilders[targetChannel].isPresent()) { - return bufferBuilders[targetChannel].get(); + if (bufferBuilders[targetChannel] != null) { + return bufferBuilders[targetChannel]; } else { return requestNewBufferBuilder(targetChannel); } @@ -101,35 +98,35 @@ public final class ChannelSelectorRecordWriter ext @Override public BufferBuilder requestNewBufferBuilder(int targetChannel) throws IOException, InterruptedException { - checkState(!bufferBuilders[targetChannel].isPresent() || bufferBuilders[targetChannel].get().isFinished()); + checkState(bufferBuilders[targetChannel] == null || bufferBuilders[targetChannel].isFinished()); BufferBuilder bufferBuilder = targetPartition.getBufferBuilder(); targetPartition.addBufferConsumer(bufferBuilder.createBufferConsumer(), targetChannel); - bufferBuilders[targetChannel] = Optional.of(bufferBuilder); + bufferBuilders[targetChannel] = bufferBuilder; return bufferBuilder; } @Override public void tryFinishCurrentBufferBuilder(int targetChannel) { - if (!bufferBuilders[targetChannel].isPresent()) { + if (bufferBuilders[targetChannel] == null) { return; } - BufferBuilder bufferBuilder = bufferBuilders[targetChannel].get(); - bufferBuilders[targetChannel] = Optional.empty(); + BufferBuilder bufferBuilder = bufferBuilders[targetChannel]; + bufferBuilders[targetChannel] = null; finishBufferBuilder(bufferBuilder); } @Override public void emptyCurrentBufferBuilder(int targetChannel) { - bufferBuilders[targetChannel] = Optional.empty(); + bufferBuilders[targetChannel] = null; } @Override public void closeBufferBuilder(int targetChannel) { - if (bufferBuilders[targetChannel].isPresent()) { - bufferBuilders[targetChannel].get().finish(); - bufferBuilders[targetChannel] = Optional.empty(); + if (bufferBuilders[targetChannel] != null) { + bufferBuilders[targetChannel].finish(); + bufferBuilders[targetChannel] = null; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java index b648eef345..0d58a8a099 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java @@ -33,8 +33,9 @@ import org.apache.flink.util.XORShiftRandom; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.IOException; -import java.util.Optional; import java.util.Random; import static org.apache.flink.runtime.io.network.api.serialization.RecordSerializer.SerializationResult; @@ -76,7 +77,8 @@ public abstract class RecordWriter { private static final String DEFAULT_OUTPUT_FLUSH_THREAD_NAME = "OutputFlusher"; /** The thread that periodically flushes the output, to give an upper latency bound. */ - private final Optional outputFlusher; + @Nullable + private final OutputFlusher outputFlusher; /** To avoid synchronization overhead on the critical path, best-effort error tracking is enough here.*/ private Throwable flusherException; @@ -90,14 +92,14 @@ public abstract class RecordWriter { checkArgument(timeout >= -1); this.flushAlways = (timeout == 0); if (timeout == -1 || timeout == 0) { - outputFlusher = Optional.empty(); + outputFlusher = null; } else { String threadName = taskName == null ? DEFAULT_OUTPUT_FLUSH_THREAD_NAME : DEFAULT_OUTPUT_FLUSH_THREAD_NAME + " for " + taskName; - outputFlusher = Optional.of(new OutputFlusher(threadName, timeout)); - outputFlusher.get().start(); + outputFlusher = new OutputFlusher(threadName, timeout); + outputFlusher.start(); } } @@ -235,10 +237,10 @@ public abstract class RecordWriter { public void close() { clearBuffers(); // make sure we terminate the thread in any case - if (outputFlusher.isPresent()) { - outputFlusher.get().terminate(); + if (outputFlusher != null) { + outputFlusher.terminate(); try { - outputFlusher.get().join(); + outputFlusher.join(); } catch (InterruptedException e) { // ignore on close // restore interrupt flag to fast exit further blocking calls -- Gitee From 64557da16b4a1b9910194a20a0aa63a7dbb95600 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Thu, 10 Oct 2019 18:43:05 +0800 Subject: [PATCH 169/268] [hotfix][runtime] Add missing generics in RecordWriterBuilder for avoiding unchecked warning --- .../api/writer/RecordWriterBuilder.java | 18 ++++++++++-------- .../iterative/task/IterationHeadTask.java | 2 +- .../SlotCountExceedingParallelismTest.java | 2 +- .../ScheduleOrUpdateConsumersTest.java | 4 ++-- .../jobmaster/TestingAbstractInvokables.java | 4 ++-- .../TaskCancelAsyncProducerConsumerITCase.java | 2 +- .../flink/runtime/jobmanager/Tasks.scala | 4 ++-- .../streaming/runtime/tasks/StreamTask.java | 2 +- .../test/runtime/FileBufferReaderITCase.java | 2 +- .../runtime/NetworkStackThroughputITCase.java | 4 ++-- 10 files changed, 23 insertions(+), 21 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterBuilder.java index 365ca2096e..b6c99b682e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterBuilder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterBuilder.java @@ -18,37 +18,39 @@ package org.apache.flink.runtime.io.network.api.writer; +import org.apache.flink.core.io.IOReadableWritable; + /** * Utility class to encapsulate the logic of building a {@link RecordWriter} instance. */ -public class RecordWriterBuilder { +public class RecordWriterBuilder { - private ChannelSelector selector = new RoundRobinChannelSelector(); + private ChannelSelector selector = new RoundRobinChannelSelector<>(); private long timeout = -1; private String taskName = "test"; - public RecordWriterBuilder setChannelSelector(ChannelSelector selector) { + public RecordWriterBuilder setChannelSelector(ChannelSelector selector) { this.selector = selector; return this; } - public RecordWriterBuilder setTimeout(long timeout) { + public RecordWriterBuilder setTimeout(long timeout) { this.timeout = timeout; return this; } - public RecordWriterBuilder setTaskName(String taskName) { + public RecordWriterBuilder setTaskName(String taskName) { this.taskName = taskName; return this; } - public RecordWriter build(ResultPartitionWriter writer) { + public RecordWriter build(ResultPartitionWriter writer) { if (selector.isBroadcast()) { - return new BroadcastRecordWriter(writer, timeout, taskName); + return new BroadcastRecordWriter<>(writer, timeout, taskName); } else { - return new ChannelSelectorRecordWriter(writer, selector, timeout, taskName); + return new ChannelSelectorRecordWriter<>(writer, selector, timeout, taskName); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadTask.java index 2ee76fcf5a..70681fd145 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationHeadTask.java @@ -143,7 +143,7 @@ public class IterationHeadTask extends AbstractIte throw new Exception("Error: Inconsistent head task setup - wrong mapping of output gates."); } // now, we can instantiate the sync gate - this.toSync = new RecordWriterBuilder().build(getEnvironment().getWriter(syncGateIndex)); + this.toSync = new RecordWriterBuilder<>().build(getEnvironment().getWriter(syncGateIndex)); this.toSyncPartitionId = getEnvironment().getWriter(syncGateIndex).getPartitionId(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java index faf2dfe924..5f3e7081ae 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java @@ -141,7 +141,7 @@ public class SlotCountExceedingParallelismTest extends TestLogger { @Override public void invoke() throws Exception { - RecordWriter writer = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); + RecordWriter writer = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); final int numberOfTimesToSend = getTaskConfiguration().getInteger(CONFIG_KEY, 0); final IntValue subtaskIndex = new IntValue( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java index 58af0e20d6..20ce3b3919 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java @@ -148,8 +148,8 @@ public class ScheduleOrUpdateConsumersTest extends TestLogger { // The order of intermediate result creation in the job graph specifies which produced // result partition is pipelined/blocking. - final RecordWriter pipelinedWriter = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); - final RecordWriter blockingWriter = new RecordWriterBuilder().build(getEnvironment().getWriter(1)); + final RecordWriter pipelinedWriter = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); + final RecordWriter blockingWriter = new RecordWriterBuilder().build(getEnvironment().getWriter(1)); writers.add(pipelinedWriter); writers.add(blockingWriter); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingAbstractInvokables.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingAbstractInvokables.java index e97dec778a..23e5dcee89 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingAbstractInvokables.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingAbstractInvokables.java @@ -47,7 +47,7 @@ public class TestingAbstractInvokables { @Override public void invoke() throws Exception { - final RecordWriter writer = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); + final RecordWriter writer = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); try { writer.emit(new IntValue(42)); @@ -97,7 +97,7 @@ public class TestingAbstractInvokables { @Override public void invoke() throws Exception { final Object o = new Object(); - RecordWriter recordWriter = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); + RecordWriter recordWriter = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); for (int i = 0; i < 1024; i++) { recordWriter.emit(new IntValue(42)); } 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 76969760d9..0d5588e78b 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 @@ -215,7 +215,7 @@ public class TaskCancelAsyncProducerConsumerITCase extends TestLogger { private final RecordWriter recordWriter; public ProducerThread(ResultPartitionWriter partitionWriter) { - this.recordWriter = new RecordWriterBuilder().build(partitionWriter); + this.recordWriter = new RecordWriterBuilder().build(partitionWriter); } @Override diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala index ea11f919db..90ee2822c3 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala @@ -35,8 +35,8 @@ object Tasks { classOf[IntValue], getEnvironment.getTaskManagerInfo.getTmpDirectories) - val writer = new RecordWriterBuilder().build( - getEnvironment.getWriter(0)).asInstanceOf[RecordWriter[IntValue]] + val writer = new RecordWriterBuilder[IntValue]().build( + getEnvironment.getWriter(0)) try { while (true) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 951fbf9678..61b1bce380 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -1413,7 +1413,7 @@ public abstract class StreamTask> } } - RecordWriter>> output = new RecordWriterBuilder() + RecordWriter>> output = new RecordWriterBuilder>>() .setChannelSelector(outputPartitioner) .setTimeout(bufferTimeout) .setTaskName(taskName) diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/FileBufferReaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/FileBufferReaderITCase.java index 49d9ef562b..91d4cf43fa 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/FileBufferReaderITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/FileBufferReaderITCase.java @@ -146,7 +146,7 @@ public class FileBufferReaderITCase extends TestLogger { @Override public void invoke() throws Exception { - final RecordWriter writer = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); + final RecordWriter writer = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); final ByteArrayType bytes = new ByteArrayType(dataSource); int counter = 0; diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java index 146cad9157..3357cb69ad 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java @@ -78,7 +78,7 @@ public class NetworkStackThroughputITCase extends TestLogger { @Override public void invoke() throws Exception { - RecordWriter writer = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); + RecordWriter writer = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); try { // Determine the amount of data to send per subtask @@ -128,7 +128,7 @@ public class NetworkStackThroughputITCase extends TestLogger { SpeedTestRecord.class, getEnvironment().getTaskManagerInfo().getTmpDirectories()); - RecordWriter writer = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); + RecordWriter writer = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); try { SpeedTestRecord record; -- Gitee From eea05611cea64c13a1c5d95a3b5e109ad7a9fb32 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 7 Jun 2019 16:09:55 +0200 Subject: [PATCH 170/268] [hotfix] Fix checkstyle violations in LeaderElectionService --- .../flink/runtime/leaderelection/LeaderElectionService.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java index c7a5588009..52b6986dbb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java @@ -25,11 +25,11 @@ import java.util.UUID; /** * Interface for a service which allows to elect a leader among a group of contenders. * - * Prior to using this service, it has to be started calling the start method. The start method + *

Prior to using this service, it has to be started calling the start method. The start method * takes the contender as a parameter. If there are multiple contenders, then each contender has * to instantiate its own leader election service. * - * Once a contender has been granted leadership he has to confirm the received leader session ID + *

Once a contender has been granted leadership he has to confirm the received leader session ID * by calling the method {@link #confirmLeadership(UUID, String)}. This will notify the leader election * service, that the contender has accepted the leadership specified and that the leader session id as * well as the leader address can now be published for leader retrieval services. -- Gitee From 4813a2396cc7a1a054b2c46c673ea96ced982523 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 7 Jun 2019 21:05:44 +0200 Subject: [PATCH 171/268] [FLINK-14298] Replace LeaderContender#getAddress with #getDescription This commit changes the LeaderContender to only require implementations to report a description of the contender used for logging purposes instead of the actual leader address. This closes #9821. --- .../nonha/embedded/EmbeddedLeaderService.java | 3 +- .../jobmaster/JobManagerRunnerImpl.java | 10 ++-- .../leaderelection/LeaderContender.java | 17 ++++--- .../ZooKeeperLeaderElectionService.java | 16 +++--- .../runtime/util/LeaderConnectionInfo.java | 2 +- .../webmonitor/WebMonitorEndpoint.java | 2 +- .../embedded/EmbeddedHaServicesTest.java | 49 +++++++------------ .../embedded/TestingLeaderContender.java | 2 +- .../SingleLeaderElectionServiceTest.java | 23 +++++---- .../leaderelection/LeaderElectionTest.java | 4 +- .../leaderelection/TestingContender.java | 4 +- .../TestingLeaderElectionService.java | 14 +++--- .../ZooKeeperLeaderElectionTest.java | 13 +++-- .../YarnIntraNonHaMasterServicesTest.java | 3 -- 14 files changed, 78 insertions(+), 84 deletions(-) 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 0cf37ddf9f..ad84cc47a4 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 @@ -299,8 +299,7 @@ public class EmbeddedLeaderService { currentLeaderProposed = leaderService; currentLeaderProposed.isLeader = true; - LOG.info("Proposing leadership to contender {} @ {}", - leaderService.contender, leaderService.contender.getAddress()); + LOG.info("Proposing leadership to contender {}", leaderService.contender.getDescription()); return execute(new GrantLeadershipCall(leaderService.contender, leaderSessionId, LOG)); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java index 52740a64f6..9b3966400f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java @@ -315,7 +315,7 @@ public class JobManagerRunnerImpl implements LeaderContender, OnCompletionAction private CompletionStage startJobMaster(UUID leaderSessionId) { log.info("JobManager runner for job {} ({}) was granted leadership with session id {} at {}.", - jobGraph.getName(), jobGraph.getJobID(), leaderSessionId, getAddress()); + jobGraph.getName(), jobGraph.getJobID(), leaderSessionId, jobMasterService.getAddress()); try { runningJobsRegistry.setJobRunning(jobGraph.getJobID()); @@ -370,7 +370,7 @@ public class JobManagerRunnerImpl implements LeaderContender, OnCompletionAction currentLeaderGatewayFuture.complete(jobMasterService.getGateway()); leaderElectionService.confirmLeadership(leaderSessionId, leaderAddress); } else { - log.debug("Ignoring confirmation of leader session id because {} is no longer the leader.", getAddress()); + log.debug("Ignoring confirmation of leader session id because {} is no longer the leader.", getDescription()); } } @@ -394,8 +394,8 @@ public class JobManagerRunnerImpl implements LeaderContender, OnCompletionAction } private CompletableFuture revokeJobMasterLeadership() { - log.info("JobManager for job {} ({}) was revoked leadership at {}.", - jobGraph.getName(), jobGraph.getJobID(), getAddress()); + log.info("JobManager for job {} ({}) at {} was revoked leadership.", + jobGraph.getName(), jobGraph.getJobID(), jobMasterService.getAddress()); setNewLeaderGatewayFuture(); @@ -431,7 +431,7 @@ public class JobManagerRunnerImpl implements LeaderContender, OnCompletionAction } @Override - public String getAddress() { + public String getDescription() { return jobMasterService.getAddress(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderContender.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderContender.java index dcf0a4e512..8b28618e93 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderContender.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderContender.java @@ -41,14 +41,6 @@ public interface LeaderContender { */ void revokeLeadership(); - /** - * Returns the address of the {@link LeaderContender} under which other instances can connect - * to it. - * - * @return Address of this contender. - */ - String getAddress(); - /** * Callback method which is called by {@link LeaderElectionService} in case of an error in the * service thread. @@ -56,4 +48,13 @@ public interface LeaderContender { * @param exception Caught exception */ void handleError(Exception exception); + + /** + * Returns the description of the {@link LeaderContender} for logging purposes. + * + * @return Description of this contender. + */ + default String getDescription() { + return "LeaderContender: " + getClass().getSimpleName(); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java index 91227dca63..382b33e2fb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java @@ -229,7 +229,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le if (LOG.isDebugEnabled()) { LOG.debug( "Grant leadership to contender {} with session ID {}.", - leaderContender.getAddress(), + leaderContender.getDescription(), issuedLeaderSessionID); } @@ -252,7 +252,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le if (running) { LOG.debug( "Revoke leadership of {} ({}@{}).", - leaderContender, + leaderContender.getDescription(), confirmedLeaderSessionID, confirmedLeaderAddress); @@ -277,7 +277,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le if (LOG.isDebugEnabled()) { LOG.debug( "Leader node changed while {} is the leader with session ID {}.", - leaderContender.getAddress(), + leaderContender.getDescription(), confirmedLeaderSessionID); } @@ -288,7 +288,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le if (LOG.isDebugEnabled()) { LOG.debug( "Writing leader information into empty node by {}.", - leaderContender.getAddress()); + leaderContender.getDescription()); } writeLeaderInformation(); } else { @@ -299,7 +299,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le if (LOG.isDebugEnabled()) { LOG.debug( "Writing leader information into node with empty data field by {}.", - leaderContender.getAddress()); + leaderContender.getDescription()); } writeLeaderInformation(); } else { @@ -315,7 +315,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le if (LOG.isDebugEnabled()) { LOG.debug( "Correcting leader information by {}.", - leaderContender.getAddress()); + leaderContender.getDescription()); } writeLeaderInformation(); } @@ -410,7 +410,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le LOG.debug("Connected to ZooKeeper quorum. Leader election can start."); break; case SUSPENDED: - LOG.warn("Connection to ZooKeeper suspended. The contender " + leaderContender.getAddress() + LOG.warn("Connection to ZooKeeper suspended. The contender " + leaderContender.getDescription() + " no longer participates in the leader election."); break; case RECONNECTED: @@ -418,7 +418,7 @@ public class ZooKeeperLeaderElectionService implements LeaderElectionService, Le break; case LOST: // Maybe we have to throw an exception here to terminate the JobManager - LOG.warn("Connection to ZooKeeper lost. The contender " + leaderContender.getAddress() + LOG.warn("Connection to ZooKeeper lost. The contender " + leaderContender.getDescription() + " no longer participates in the leader election."); break; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderConnectionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderConnectionInfo.java index 3c9ed142a5..c58691989d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderConnectionInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderConnectionInfo.java @@ -28,7 +28,7 @@ public class LeaderConnectionInfo { private final String address; - LeaderConnectionInfo(UUID leaderSessionId, String address) { + public LeaderConnectionInfo(UUID leaderSessionId, String address) { this.leaderSessionId = leaderSessionId; this.address = address; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index f474ad50df..6b1ff00e8c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -719,7 +719,7 @@ public class WebMonitorEndpoint extends RestServerEndp } @Override - public String getAddress() { + public String getDescription() { return getRestBaseUrl(); } 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 3b9ce6513b..840d96cf51 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 @@ -22,8 +22,9 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.util.LeaderConnectionInfo; +import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.apache.flink.util.TestLogger; import org.junit.After; @@ -37,17 +38,17 @@ import static junit.framework.TestCase.assertTrue; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; /** * Tests for the {@link EmbeddedHaServices}. */ public class EmbeddedHaServicesTest extends TestLogger { + private static final String ADDRESS = "foobar"; + private EmbeddedHaServices embeddedHaServices; @Before @@ -120,26 +121,29 @@ public class EmbeddedHaServicesTest extends TestLogger { */ @Test public void testJobManagerLeaderRetrieval() throws Exception { - final String address = "foobar"; JobID jobId = new JobID(); - LeaderRetrievalListener leaderRetrievalListener = mock(LeaderRetrievalListener.class); - LeaderContender leaderContender = mock(LeaderContender.class); - when(leaderContender.getAddress()).thenReturn(address); LeaderElectionService leaderElectionService = embeddedHaServices.getJobManagerLeaderElectionService(jobId); LeaderRetrievalService leaderRetrievalService = embeddedHaServices.getJobManagerLeaderRetriever(jobId); + runLeaderRetrievalTest(leaderElectionService, leaderRetrievalService); + } + + private void runLeaderRetrievalTest(LeaderElectionService leaderElectionService, LeaderRetrievalService leaderRetrievalService) throws Exception { + LeaderRetrievalUtils.LeaderConnectionInfoListener leaderRetrievalListener = new LeaderRetrievalUtils.LeaderConnectionInfoListener(); + TestingLeaderContender leaderContender = new TestingLeaderContender(); + leaderRetrievalService.start(leaderRetrievalListener); leaderElectionService.start(leaderContender); - ArgumentCaptor leaderIdArgumentCaptor = ArgumentCaptor.forClass(UUID.class); - verify(leaderContender).grantLeadership(leaderIdArgumentCaptor.capture()); + final UUID leaderId = leaderContender.getLeaderSessionFuture().get(); - final UUID leaderId = leaderIdArgumentCaptor.getValue(); + leaderElectionService.confirmLeadership(leaderId, ADDRESS); - leaderElectionService.confirmLeadership(leaderId, address); + final LeaderConnectionInfo leaderConnectionInfo = leaderRetrievalListener.getLeaderConnectionInfoFuture().get(); - verify(leaderRetrievalListener).notifyLeaderAddress(eq(address), eq(leaderId)); + assertThat(leaderConnectionInfo.getAddress(), is(ADDRESS)); + assertThat(leaderConnectionInfo.getLeaderSessionId(), is(leaderId)); } /** @@ -147,25 +151,10 @@ public class EmbeddedHaServicesTest extends TestLogger { */ @Test public void testResourceManagerLeaderRetrieval() throws Exception { - final String address = "foobar"; - LeaderRetrievalListener leaderRetrievalListener = mock(LeaderRetrievalListener.class); - LeaderContender leaderContender = mock(LeaderContender.class); - when(leaderContender.getAddress()).thenReturn(address); - LeaderElectionService leaderElectionService = embeddedHaServices.getResourceManagerLeaderElectionService(); LeaderRetrievalService leaderRetrievalService = embeddedHaServices.getResourceManagerLeaderRetriever(); - leaderRetrievalService.start(leaderRetrievalListener); - leaderElectionService.start(leaderContender); - - ArgumentCaptor leaderIdArgumentCaptor = ArgumentCaptor.forClass(UUID.class); - verify(leaderContender).grantLeadership(leaderIdArgumentCaptor.capture()); - - final UUID leaderId = leaderIdArgumentCaptor.getValue(); - - leaderElectionService.confirmLeadership(leaderId, address); - - verify(leaderRetrievalListener).notifyLeaderAddress(eq(address), eq(leaderId)); + runLeaderRetrievalTest(leaderElectionService, leaderRetrievalService); } /** @@ -191,8 +180,8 @@ public class EmbeddedHaServicesTest extends TestLogger { assertThat(dispatcherLeaderElectionService.hasLeadership(newLeaderSessionId), is(true)); - dispatcherLeaderElectionService.confirmLeadership(oldLeaderSessionId, leaderContender.getAddress()); - dispatcherLeaderElectionService.confirmLeadership(newLeaderSessionId, leaderContender.getAddress()); + dispatcherLeaderElectionService.confirmLeadership(oldLeaderSessionId, ADDRESS); + dispatcherLeaderElectionService.confirmLeadership(newLeaderSessionId, ADDRESS); assertThat(dispatcherLeaderElectionService.hasLeadership(newLeaderSessionId), is(true)); 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 index 0b28671676..9bad812875 100644 --- 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 @@ -58,7 +58,7 @@ final class TestingLeaderContender implements LeaderContender { } @Override - public String getAddress() { + public String getDescription() { return "foobar"; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java index 535679ea65..910db15999 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java @@ -26,7 +26,6 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.util.StringUtils; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import java.util.Random; @@ -34,7 +33,12 @@ import java.util.UUID; import java.util.concurrent.Executor; import static org.junit.Assert.fail; -import static org.mockito.Mockito.*; +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.verifyNoMoreInteractions; /** * Tests for the {@link SingleLeaderElectionService}. @@ -165,7 +169,7 @@ public class SingleLeaderElectionServiceTest { service.shutdown(); final LeaderContender contender = mock(LeaderContender.class); - + // should not be possible to start try { service.start(contender); @@ -210,15 +214,10 @@ public class SingleLeaderElectionServiceTest { private static LeaderContender mockContender(final LeaderElectionService service, final String address) { LeaderContender mockContender = mock(LeaderContender.class); - when(mockContender.getAddress()).thenReturn(address); - - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - final UUID uuid = (UUID) invocation.getArguments()[0]; - service.confirmLeadership(uuid, address); - return null; - } + doAnswer((Answer) invocation -> { + final UUID uuid = (UUID) invocation.getArguments()[0]; + service.confirmLeadership(uuid, address); + return null; }).when(mockContender).grantLeadership(any(UUID.class)); return mockContender; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionTest.java index 9090cae6c2..a1d99cdb4f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionTest.java @@ -101,7 +101,7 @@ public class LeaderElectionTest extends TestLogger { assertThat(leaderElectionService.hasLeadership(leaderSessionId), is(true)); assertThat(leaderElectionService.hasLeadership(UUID.randomUUID()), is(false)); - leaderElectionService.confirmLeadership(leaderSessionId, manualLeaderContender.getAddress()); + leaderElectionService.confirmLeadership(leaderSessionId, "foobar"); assertThat(leaderElectionService.hasLeadership(leaderSessionId), is(true)); @@ -132,7 +132,7 @@ public class LeaderElectionTest extends TestLogger { } @Override - public String getAddress() { + public String getDescription() { return "foobar"; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingContender.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingContender.java index 34408ff2ea..3775d99635 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingContender.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingContender.java @@ -118,7 +118,7 @@ public class TestingContender implements LeaderContender { this.leaderSessionID = leaderSessionID; - leaderElectionService.confirmLeadership(leaderSessionID, getAddress()); + leaderElectionService.confirmLeadership(leaderSessionID, address); leader = true; @@ -139,7 +139,7 @@ public class TestingContender implements LeaderContender { } @Override - public String getAddress() { + public String getDescription() { return address; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java index f282ebdc4d..8d25f609b0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.leaderelection; +import org.apache.flink.runtime.util.LeaderConnectionInfo; + import javax.annotation.Nonnull; import java.util.UUID; @@ -31,7 +33,7 @@ public class TestingLeaderElectionService implements LeaderElectionService { private LeaderContender contender = null; private boolean hasLeadership = false; - private CompletableFuture confirmationFuture = null; + private CompletableFuture confirmationFuture = null; private CompletableFuture startFuture = new CompletableFuture<>(); private UUID issuedLeaderSessionId = null; @@ -40,7 +42,7 @@ public class TestingLeaderElectionService implements LeaderElectionService { * *

If a job graph with the same {@link JobID} exists, it is replaced. - */ - void putJobGraph(JobGraph jobGraph) throws Exception; - - /** - * Removes the {@link JobGraph} with the given {@link JobID} if it exists. - */ - void removeJobGraph(JobID jobId) throws Exception; - - /** - * Releases the locks on the specified {@link JobGraph}. - * - * Releasing the locks allows that another instance can delete the job from - * the {@link JobGraphStore}. - * - * @param jobId specifying the job to release the locks for - * @throws Exception if the locks cannot be released - */ - void releaseJobGraph(JobID jobId) throws Exception; - /** * Get all job ids of submitted job graphs to the submitted job graph store. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphWriter.java new file mode 100644 index 0000000000..7a987952a9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphWriter.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobGraph; + +/** + * Allows to store and remove job graphs. + */ +public interface JobGraphWriter { + /** + * Adds the {@link JobGraph} instance. + * + *

If a job graph with the same {@link JobID} exists, it is replaced. + */ + void putJobGraph(JobGraph jobGraph) throws Exception; + + /** + * Removes the {@link JobGraph} with the given {@link JobID} if it exists. + */ + void removeJobGraph(JobID jobId) throws Exception; + + /** + * Releases the locks on the specified {@link JobGraph}. + * + * Releasing the locks allows that another instance can delete the job from + * the {@link JobGraphStore}. + * + * @param jobId specifying the job to release the locks for + * @throws Exception if the locks cannot be released + */ + void releaseJobGraph(JobID jobId) throws Exception; +} -- Gitee From 2c2095bdad3d47f27973a585112ed820f457de6f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 11 Oct 2019 11:28:54 +0200 Subject: [PATCH 179/268] [hotfix][e2e] Disable Avro Confluent schema registry e2e test until FLINK-13567 has been fixed --- tools/travis/splits/split_misc.sh | 3 ++- tools/travis/splits/split_misc_hadoopfree.sh | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tools/travis/splits/split_misc.sh b/tools/travis/splits/split_misc.sh index 73fa7682ea..834c863850 100755 --- a/tools/travis/splits/split_misc.sh +++ b/tools/travis/splits/split_misc.sh @@ -66,7 +66,8 @@ run_test "Elasticsearch (v6.3.1) sink end-to-end test" "$END_TO_END_DIR/test-scr 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" +# Disabled until FLINK-13567 has been fixed +#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" diff --git a/tools/travis/splits/split_misc_hadoopfree.sh b/tools/travis/splits/split_misc_hadoopfree.sh index 193ad0b13a..e4a36a8f96 100755 --- a/tools/travis/splits/split_misc_hadoopfree.sh +++ b/tools/travis/splits/split_misc_hadoopfree.sh @@ -63,7 +63,8 @@ run_test "Elasticsearch (v6.3.1) sink end-to-end test" "$END_TO_END_DIR/test-scr 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" +# Disabled until FLINK-13567 has been fixed +#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" -- Gitee From 1c7472547555bf7b71344516c45fc7063d2f859a Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 10 Oct 2019 14:44:13 +0200 Subject: [PATCH 180/268] [hotfix][runtime, test] Unify formatting in StreamConfigChainer --- .../runtime/tasks/StreamConfigChainer.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java index 06e1b99ead..eaa86c79b8 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java @@ -61,10 +61,10 @@ public class StreamConfigChainer { } public StreamConfigChainer chain( - OperatorID operatorID, - OneInputStreamOperator operator, - TypeSerializer typeSerializer, - boolean createKeyedStateBackend) { + OperatorID operatorID, + OneInputStreamOperator operator, + TypeSerializer typeSerializer, + boolean createKeyedStateBackend) { return chain(operatorID, operator, typeSerializer, typeSerializer, createKeyedStateBackend); } @@ -76,10 +76,10 @@ public class StreamConfigChainer { } public StreamConfigChainer chain( - OperatorID operatorID, - OneInputStreamOperator operator, - TypeSerializer inputSerializer, - TypeSerializer outputSerializer) { + OperatorID operatorID, + OneInputStreamOperator operator, + TypeSerializer inputSerializer, + TypeSerializer outputSerializer) { return chain(operatorID, operator, inputSerializer, outputSerializer, false); } -- Gitee From db6fdea7e68e95e309c3cb0999ab9a5cc58f141a Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 10 Oct 2019 14:30:33 +0200 Subject: [PATCH 181/268] [hotfix][runtime, test] Refactor StreamConfigChainer to support StreamOperatorFactory and honour pass down the bufferTimeout --- .../runtime/tasks/StreamConfigChainer.java | 34 +++++++++++++++---- .../runtime/tasks/StreamTaskTestHarness.java | 2 +- 2 files changed, 29 insertions(+), 7 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java index eaa86c79b8..0efb278946 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamConfigChainer.java @@ -25,7 +25,9 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.graph.StreamEdge; import org.apache.flink.streaming.api.graph.StreamNode; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; import java.util.Collections; @@ -42,22 +44,29 @@ import static org.apache.flink.util.Preconditions.checkNotNull; public class StreamConfigChainer { private final StreamConfig headConfig; private final Map chainedConfigs = new HashMap<>(); + private final long bufferTimeout; private StreamConfig tailConfig; private int chainIndex = 0; public StreamConfigChainer(OperatorID headOperatorID, StreamOperator headOperator, StreamConfig headConfig) { + this(headOperatorID, SimpleOperatorFactory.of(headOperator), headConfig); + } + + public StreamConfigChainer(OperatorID headOperatorID, StreamOperatorFactory headOperatorFactory, StreamConfig headConfig) { this.headConfig = checkNotNull(headConfig); this.tailConfig = checkNotNull(headConfig); + this.bufferTimeout = headConfig.getBufferTimeout(); - head(headOperator, headOperatorID); + head(headOperatorID, headOperatorFactory); } - private void head(StreamOperator headOperator, OperatorID headOperatorID) { - headConfig.setStreamOperator(headOperator); + private void head(OperatorID headOperatorID, StreamOperatorFactory headOperatorFactory) { + headConfig.setStreamOperatorFactory(headOperatorFactory); headConfig.setOperatorID(headOperatorID); headConfig.setChainStart(); headConfig.setChainIndex(chainIndex); + headConfig.setBufferTimeout(bufferTimeout); } public StreamConfigChainer chain( @@ -89,6 +98,20 @@ public class StreamConfigChainer { TypeSerializer inputSerializer, TypeSerializer outputSerializer, boolean createKeyedStateBackend) { + return chain( + operatorID, + SimpleOperatorFactory.of(operator), + inputSerializer, + outputSerializer, + createKeyedStateBackend); + } + + public StreamConfigChainer chain( + OperatorID operatorID, + StreamOperatorFactory operatorFactory, + TypeSerializer inputSerializer, + TypeSerializer outputSerializer, + boolean createKeyedStateBackend) { chainIndex++; tailConfig.setChainedOutputs(Collections.singletonList( @@ -100,7 +123,7 @@ public class StreamConfigChainer { null, null))); tailConfig = new StreamConfig(new Configuration()); - tailConfig.setStreamOperator(checkNotNull(operator)); + tailConfig.setStreamOperatorFactory(checkNotNull(operatorFactory)); tailConfig.setOperatorID(checkNotNull(operatorID)); tailConfig.setTypeSerializerIn1(inputSerializer); tailConfig.setTypeSerializerOut(outputSerializer); @@ -109,6 +132,7 @@ public class StreamConfigChainer { tailConfig.setStateKeySerializer(inputSerializer); } tailConfig.setChainIndex(chainIndex); + tailConfig.setBufferTimeout(bufferTimeout); chainedConfigs.put(chainIndex, tailConfig); @@ -116,7 +140,6 @@ public class StreamConfigChainer { } public void finish() { - List outEdgesInOrder = new LinkedList(); outEdgesInOrder.add( new StreamEdge( @@ -127,7 +150,6 @@ public class StreamConfigChainer { new BroadcastPartitioner(), null)); - tailConfig.setBufferTimeout(0); tailConfig.setChainEnd(); tailConfig.setOutputSelectors(Collections.emptyList()); tailConfig.setNumberOfOutputs(1); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java index aa5f25e0a4..c5db3ffff7 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java @@ -136,6 +136,7 @@ public class StreamTaskTestHarness { this.executionConfig = new ExecutionConfig(); streamConfig = new StreamConfig(taskConfig); + streamConfig.setBufferTimeout(0); outputSerializer = outputType.createSerializer(executionConfig); outputStreamRecordSerializer = new StreamElementSerializer(outputSerializer); @@ -184,7 +185,6 @@ public class StreamTaskTestHarness { Preconditions.checkState(!setupCalled, "This harness was already setup."); setupCalled = true; streamConfig.setChainStart(); - streamConfig.setBufferTimeout(0); streamConfig.setTimeCharacteristic(TimeCharacteristic.EventTime); streamConfig.setOutputSelectors(Collections.>emptyList()); streamConfig.setNumberOfOutputs(1); -- Gitee From 0bd0ec17040fa9deaa16754dc74693307a89cb3e Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 10 Oct 2019 14:31:53 +0200 Subject: [PATCH 182/268] [FLINK-14300][runtime, test] Add test making sure that RecordWriter is properly closed in case of early StreamTask failure This adds a missing test coverage for the 0d112f5bc61e6f8400e000e13add08abae1067a1 commit. --- .../io/network/api/writer/RecordWriter.java | 8 +- .../streaming/api/graph/StreamConfig.java | 4 +- .../runtime/tasks/StreamTaskTest.java | 76 +++++++++++++++++++ 3 files changed, 84 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java index 0d58a8a099..93fa826305 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.api.writer; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.SimpleCounter; @@ -57,6 +58,10 @@ import static org.apache.flink.util.Preconditions.checkState; */ public abstract class RecordWriter { + /** Default name for the output flush thread, if no name with a task reference is given. */ + @VisibleForTesting + public static final String DEFAULT_OUTPUT_FLUSH_THREAD_NAME = "OutputFlusher"; + private static final Logger LOG = LoggerFactory.getLogger(RecordWriter.class); protected final ResultPartitionWriter targetPartition; @@ -73,9 +78,6 @@ public abstract class RecordWriter { private final boolean flushAlways; - /** Default name for teh output flush thread, if no name with a task reference is given. */ - private static final String DEFAULT_OUTPUT_FLUSH_THREAD_NAME = "OutputFlusher"; - /** The thread that periodically flushes the output, to give an upper latency bound. */ @Nullable private final OutputFlusher outputFlusher; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index a4f83c1af8..5384541030 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -58,6 +58,9 @@ public class StreamConfig implements Serializable { // Config Keys // ------------------------------------------------------------------------ + @VisibleForTesting + public static final String SERIALIZEDUDF = "serializedUDF"; + private static final String NUMBER_OF_OUTPUTS = "numberOfOutputs"; private static final String NUMBER_OF_INPUTS = "numberOfInputs"; private static final String CHAINED_OUTPUTS = "chainedOutputs"; @@ -67,7 +70,6 @@ public class StreamConfig implements Serializable { private static final String VERTEX_NAME = "vertexID"; private static final String ITERATION_ID = "iterationId"; private static final String OUTPUT_SELECTOR_WRAPPER = "outputSelectorWrapper"; - private static final String SERIALIZEDUDF = "serializedUDF"; private static final String BUFFER_TIMEOUT = "bufferTimeout"; private static final String TYPE_SERIALIZER_IN_1 = "typeSerializer_in_1"; private static final String TYPE_SERIALIZER_IN_2 = "typeSerializer_in_2"; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 297c0640dc..222e45b70f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -21,6 +21,7 @@ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; @@ -53,6 +54,7 @@ import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.TaskEventDispatcher; +import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -102,10 +104,12 @@ import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.InternalTimeServiceManager; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperatorStateContext; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; @@ -135,6 +139,7 @@ import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.io.ObjectInputStream; +import java.io.StreamCorruptedException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -836,6 +841,53 @@ public class StreamTaskTest extends TestLogger { } } + /** + * This test ensures that {@link RecordWriter} is correctly closed even if we fail to construct + * {@link OperatorChain}, for example because of user class deserialization error. + */ + @Test + public void testRecordWriterClosedOnStreamOperatorFactoryDeserializationError() throws Exception { + Configuration taskConfiguration = new Configuration(); + StreamConfig streamConfig = new StreamConfig(taskConfiguration); + + // Make sure that there is some output edge in the config so that some RecordWriter is created + StreamConfigChainer cfg = new StreamConfigChainer(new OperatorID(42, 42), new UnusedOperatorFactory(), streamConfig); + cfg.chain( + new OperatorID(44, 44), + new UnusedOperatorFactory(), + StringSerializer.INSTANCE, + StringSerializer.INSTANCE, + false); + cfg.finish(); + + // Overwrite the serialized bytes to some garbage to induce deserialization exception + taskConfiguration.setBytes(StreamConfig.SERIALIZEDUDF, new byte[42]); + + try (MockEnvironment mockEnvironment = + new MockEnvironmentBuilder() + .setTaskConfiguration(taskConfiguration) + .build()) { + + mockEnvironment.addOutput(new ArrayList<>()); + StreamTask streamTask = + new NoOpStreamTask<>(mockEnvironment); + + try { + streamTask.invoke(); + fail("Should have failed with an exception!"); + } catch (Exception ex) { + if (!ExceptionUtils.findThrowable(ex, StreamCorruptedException.class).isPresent()) { + throw ex; + } + } + } + + assertTrue( + RecordWriter.DEFAULT_OUTPUT_FLUSH_THREAD_NAME + " thread is still running", + Thread.getAllStackTraces().keySet().stream() + .noneMatch(thread -> thread.getName().startsWith(RecordWriter.DEFAULT_OUTPUT_FLUSH_THREAD_NAME))); + } + // ------------------------------------------------------------------------ // Test Utilities // ------------------------------------------------------------------------ @@ -1695,4 +1747,28 @@ public class StreamTaskTest extends TestLogger { return lastDeclinedCheckpointCause; } } + + private static class UnusedOperatorFactory implements StreamOperatorFactory { + @Override + public > T createStreamOperator( + StreamTask containingTask, + StreamConfig config, + Output> output) { + throw new UnsupportedOperationException("This shouldn't be called"); + } + + @Override + public void setChainingStrategy(ChainingStrategy strategy) { + } + + @Override + public ChainingStrategy getChainingStrategy() { + return ChainingStrategy.ALWAYS; + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + throw new UnsupportedOperationException(); + } + } } -- Gitee From fbfa8beb6847ed14641399afbbfd9a378d91e6f5 Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 11 Oct 2019 10:10:43 +0800 Subject: [PATCH 183/268] [FLINK-14347][test] Filter out expected exception string in YARN tests This closes #9880. --- .../src/test/java/org/apache/flink/yarn/YarnTestBase.java | 3 +++ 1 file changed, 3 insertions(+) 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 3381f10544..6481420d20 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 @@ -131,6 +131,9 @@ public abstract class YarnTestBase extends TestLogger { "Remote connection to [null] failed with java.nio.channels.NotYetConnectedException", "java.io.IOException: Connection reset by peer", + // filter out expected ResourceManagerException caused by intended shutdown request + YarnResourceManager.ERROR_MASSAGE_ON_SHUTDOWN_REQUEST, + // this can happen in Akka 2.4 on shutdown. "java.util.concurrent.RejectedExecutionException: Worker has already been shutdown", -- Gitee From 80b27a150026b7b5cb707bd9fa3e17f565bb8112 Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 8 Oct 2019 16:09:04 +0800 Subject: [PATCH 184/268] [FLINK-14343][coordination] Remove uncompleted YARNHighAvailabilityService This closes #9852. --- .../FsNegativeRunningJobsRegistryTest.java | 134 ------ .../FsNegativeRunningJobsRegistry.java | 185 --------- .../LeaderIdMismatchException.java | 47 --- .../ServicesThreadFactory.java | 40 -- .../SingleLeaderElectionService.java | 389 ------------------ .../FsNegativeRunningJobsRegistryTest.java | 95 ----- .../SingleLeaderElectionServiceTest.java | 225 ---------- .../AbstractYarnNonHaServices.java | 93 ----- .../YarnHighAvailabilityServices.java | 371 ----------------- .../YarnIntraNonHaMasterServices.java | 246 ----------- .../YarnPreConfiguredMasterNonHaServices.java | 243 ----------- .../YarnIntraNonHaMasterServicesTest.java | 163 -------- ...YarnPreConfiguredMasterHaServicesTest.java | 225 ---------- 13 files changed, 2456 deletions(-) delete mode 100644 flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FsNegativeRunningJobsRegistryTest.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/LeaderIdMismatchException.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ServicesThreadFactory.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionService.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java delete mode 100644 flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/AbstractYarnNonHaServices.java delete mode 100644 flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnHighAvailabilityServices.java delete mode 100644 flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java delete mode 100644 flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java delete mode 100644 flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java delete mode 100644 flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterHaServicesTest.java diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FsNegativeRunningJobsRegistryTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FsNegativeRunningJobsRegistryTest.java deleted file mode 100644 index 1273a4ee9d..0000000000 --- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FsNegativeRunningJobsRegistryTest.java +++ /dev/null @@ -1,134 +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.hdfstests; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.core.fs.Path; -import org.apache.flink.runtime.highavailability.FsNegativeRunningJobsRegistry; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus; -import org.apache.flink.util.OperatingSystem; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.AfterClass; -import org.junit.Assume; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.File; - -import static org.junit.Assert.assertEquals; - -/** - * Tests for the {@link FsNegativeRunningJobsRegistry} on HDFS. - */ -public class FsNegativeRunningJobsRegistryTest { - - @ClassRule - public static final TemporaryFolder TEMP_DIR = new TemporaryFolder(); - - private static MiniDFSCluster hdfsCluster; - - private static Path hdfsRootPath; - - // ------------------------------------------------------------------------ - // startup / shutdown - // ------------------------------------------------------------------------ - - @BeforeClass - public static void createHDFS() throws Exception { - Assume.assumeTrue("HDFS cluster cannot be start on Windows without extensions.", !OperatingSystem.isWindows()); - - final File tempDir = TEMP_DIR.newFolder(); - - Configuration hdConf = new Configuration(); - hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, tempDir.getAbsolutePath()); - - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf); - hdfsCluster = builder.build(); - - hdfsRootPath = new Path("hdfs://" + hdfsCluster.getURI().getHost() + ":" - + hdfsCluster.getNameNodePort() + "/"); - } - - @AfterClass - public static void destroyHDFS() { - if (hdfsCluster != null) { - hdfsCluster.shutdown(); - } - hdfsCluster = null; - hdfsRootPath = null; - } - - // ------------------------------------------------------------------------ - // Tests - // ------------------------------------------------------------------------ - - @Test - public void testCreateAndSetFinished() throws Exception { - final Path workDir = new Path(hdfsRootPath, "test-work-dir"); - final JobID jid = new JobID(); - - FsNegativeRunningJobsRegistry registry = new FsNegativeRunningJobsRegistry(workDir); - - // another registry should pick this up - FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(workDir); - - // initially, without any call, the job is pending - assertEquals(JobSchedulingStatus.PENDING, registry.getJobSchedulingStatus(jid)); - assertEquals(JobSchedulingStatus.PENDING, otherRegistry.getJobSchedulingStatus(jid)); - - // after set running, the job is running - registry.setJobRunning(jid); - assertEquals(JobSchedulingStatus.RUNNING, registry.getJobSchedulingStatus(jid)); - assertEquals(JobSchedulingStatus.RUNNING, otherRegistry.getJobSchedulingStatus(jid)); - - // set the job to finished and validate - registry.setJobFinished(jid); - assertEquals(JobSchedulingStatus.DONE, registry.getJobSchedulingStatus(jid)); - assertEquals(JobSchedulingStatus.DONE, otherRegistry.getJobSchedulingStatus(jid)); - } - - @Test - public void testSetFinishedAndRunning() throws Exception { - final Path workDir = new Path(hdfsRootPath, "änother_wörk_directörü"); - final JobID jid = new JobID(); - - FsNegativeRunningJobsRegistry registry = new FsNegativeRunningJobsRegistry(workDir); - - // set the job to finished and validate - registry.setJobFinished(jid); - assertEquals(JobSchedulingStatus.DONE, registry.getJobSchedulingStatus(jid)); - - // set the job to running does not overwrite the finished status - registry.setJobRunning(jid); - assertEquals(JobSchedulingStatus.DONE, registry.getJobSchedulingStatus(jid)); - - // another registry should pick this up - FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(workDir); - assertEquals(JobSchedulingStatus.DONE, otherRegistry.getJobSchedulingStatus(jid)); - - // clear the running and finished marker, it will be pending - otherRegistry.clearJob(jid); - assertEquals(JobSchedulingStatus.PENDING, registry.getJobSchedulingStatus(jid)); - assertEquals(JobSchedulingStatus.PENDING, otherRegistry.getJobSchedulingStatus(jid)); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java deleted file mode 100644 index d29dcec47a..0000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistry.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.highavailability; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.core.fs.FSDataOutputStream; -import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.core.fs.FileSystem.WriteMode; -import org.apache.flink.core.fs.Path; - -import java.io.FileNotFoundException; -import java.io.IOException; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * This {@link RunningJobsRegistry} tracks the status jobs via marker files, - * marking running jobs via running marker files, marking finished jobs via finished marker files. - * - *

The general contract is the following: - *

    - *
  • Initially, a marker file does not exist (no one created it, yet), which means - * the specific job is pending.
  • - *
  • The first JobManager that granted leadership calls this service to create the running marker file, - * which marks the job as running.
  • - *
  • If a JobManager gains leadership but sees the running marker file, - * it will realize that the job has been scheduled already and needs reconciling.
  • - *
  • The JobManager that finishes calls this service to create the marker file, - * which marks the job as finished.
  • - *
  • If a JobManager gains leadership at some point when shutdown is in progress, - * it will see the marker file and realize that the job is finished.
  • - *
  • The application framework is expected to clean the file once the application - * is completely shut down. At that point, no JobManager will attempt to - * start the job, even if it gains leadership.
  • - *
- * - *

This registry is especially tailored towards deployment modes like for example - * YARN, where HDFS is available as a persistent file system, and the YARN - * application's working directories on HDFS are automatically cleaned - * up after the application completed. - */ -public class FsNegativeRunningJobsRegistry implements RunningJobsRegistry { - - private static final String DONE_PREFIX = ".job_complete_"; - - private static final String RUNNING_PREFIX = ".job_runing_"; - - private final FileSystem fileSystem; - - private final Path basePath; - - /** - * Creates a new registry that writes to the FileSystem and working directory - * denoted by the given path. - * - *

The initialization will attempt to write to the given working directory, in - * order to catch setup/configuration errors early. - * - * @param workingDirectory The working directory for files to track the job status. - * - * @throws IOException Thrown, if the specified directory cannot be accessed. - */ - public FsNegativeRunningJobsRegistry(Path workingDirectory) throws IOException { - this(workingDirectory.getFileSystem(), workingDirectory); - } - - /** - * Creates a new registry that writes its files to the given FileSystem at - * the given working directory path. - * - *

The initialization will attempt to write to the given working directory, in - * order to catch setup/configuration errors early. - * - * @param fileSystem The FileSystem to use for the marker files. - * @param workingDirectory The working directory for files to track the job status. - * - * @throws IOException Thrown, if the specified directory cannot be accessed. - */ - public FsNegativeRunningJobsRegistry(FileSystem fileSystem, Path workingDirectory) throws IOException { - this.fileSystem = checkNotNull(fileSystem, "fileSystem"); - this.basePath = checkNotNull(workingDirectory, "workingDirectory"); - - // to be safe, attempt to write to the working directory, to - // catch problems early - final Path testFile = new Path(workingDirectory, ".registry_test"); - try { - createFile(testFile, false); - } - catch (IOException e) { - throw new IOException("Unable to write to working directory: " + workingDirectory, e); - } - finally { - fileSystem.delete(testFile, false); - } - } - - // ------------------------------------------------------------------------ - - @Override - public void setJobRunning(JobID jobID) throws IOException { - checkNotNull(jobID, "jobID"); - final Path filePath = createMarkerFilePath(RUNNING_PREFIX, jobID); - - // create the file - // to avoid an exception if the job already exists, set overwrite=true - createFile(filePath, true); - } - - @Override - public void setJobFinished(JobID jobID) throws IOException { - checkNotNull(jobID, "jobID"); - final Path filePath = createMarkerFilePath(DONE_PREFIX, jobID); - - // create the file - // to avoid an exception if the job already exists, set overwrite=true - createFile(filePath, true); - } - - @Override - public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException { - checkNotNull(jobID, "jobID"); - - // first check for the existence of the complete file - if (fileSystem.exists(createMarkerFilePath(DONE_PREFIX, jobID))) { - // complete file was found --> job is terminated - return JobSchedulingStatus.DONE; - } - // check for the existence of the running file - else if (fileSystem.exists(createMarkerFilePath(RUNNING_PREFIX, jobID))) { - // running file was found --> job is terminated - return JobSchedulingStatus.RUNNING; - } - else { - // file does not exist, job is not scheduled - return JobSchedulingStatus.PENDING; - } - } - - @Override - public void clearJob(JobID jobID) throws IOException { - checkNotNull(jobID, "jobID"); - final Path runningFilePath = createMarkerFilePath(RUNNING_PREFIX, jobID); - final Path doneFilePath = createMarkerFilePath(DONE_PREFIX, jobID); - - // delete the running marker file, if it exists - try { - fileSystem.delete(runningFilePath, false); - } - catch (FileNotFoundException ignored) {} - - // delete the finished marker file, if it exists - try { - fileSystem.delete(doneFilePath, false); - } - catch (FileNotFoundException ignored) {} - } - - private Path createMarkerFilePath(String prefix, JobID jobId) { - return new Path(basePath, prefix + jobId.toString()); - } - - private void createFile(Path path, boolean overwrite) throws IOException { - final WriteMode writeMode = overwrite ? WriteMode.OVERWRITE : WriteMode.NO_OVERWRITE; - - try (FSDataOutputStream out = fileSystem.create(path, writeMode)) { - out.write(42); - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/LeaderIdMismatchException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/LeaderIdMismatchException.java deleted file mode 100644 index 5caf1b296b..0000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/LeaderIdMismatchException.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.runtime.highavailability; - -import java.util.UUID; - -/** - * An exception thrown when the leader session id attached to a message does not match - * the leader session id that the receiver expects. - */ -public class LeaderIdMismatchException extends Exception { - - private static final long serialVersionUID = 1L; - - private final UUID expected; - private final UUID actual; - - public LeaderIdMismatchException(UUID expected, UUID actual) { - super("Leader session ID mismatch: expected=" + expected + ", actual=" + actual); - this.expected = expected; - this.actual = actual; - } - - public UUID getExpected() { - return expected; - } - - public UUID getActual() { - return actual; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ServicesThreadFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ServicesThreadFactory.java deleted file mode 100644 index 24667e4252..0000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ServicesThreadFactory.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.highavailability; - -import javax.annotation.Nonnull; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.atomic.AtomicInteger; - -public class ServicesThreadFactory implements ThreadFactory { - - private AtomicInteger enumerator = new AtomicInteger(); - - @Override - public Thread newThread(@Nonnull Runnable r) { - Thread thread = new Thread(r, "Flink HA Services Thread #" + enumerator.incrementAndGet()); - - // HA threads should have a very high priority, but not - // keep the JVM running by themselves - thread.setPriority(Thread.MAX_PRIORITY); - thread.setDaemon(true); - - return thread; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionService.java deleted file mode 100644 index 879b52356c..0000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionService.java +++ /dev/null @@ -1,389 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.highavailability.nonha.leaderelection; - -import org.apache.flink.runtime.leaderelection.LeaderContender; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; -import javax.annotation.concurrent.GuardedBy; - -import java.util.HashSet; -import java.util.UUID; -import java.util.concurrent.Executor; - -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; - -/** - * An implementation of the {@link LeaderElectionService} interface that handles a single - * leader contender. When started, this service immediately grants the contender the leadership. - * - *

The implementation accepts a single static leader session ID and is hence compatible with - * pre-configured single leader (no leader failover) setups. - * - *

This implementation supports a series of leader listeners that receive notifications about - * the leader contender. - */ -public class SingleLeaderElectionService implements LeaderElectionService { - - private static final Logger LOG = LoggerFactory.getLogger(SingleLeaderElectionService.class); - - // ------------------------------------------------------------------------ - - /** lock for all operations on this instance. */ - private final Object lock = new Object(); - - /** The executor service that dispatches notifications. */ - private final Executor notificationExecutor; - - /** The leader ID assigned to the immediate leader. */ - private final UUID leaderId; - - @GuardedBy("lock") - private final HashSet listeners; - - /** The currently proposed leader. */ - @GuardedBy("lock") - private volatile LeaderContender proposedLeader; - - /** The confirmed leader. */ - @GuardedBy("lock") - private volatile LeaderContender leader; - - /** The address of the confirmed leader. */ - @GuardedBy("lock") - private volatile String leaderAddress; - - /** Flag marking this service as shutdown, meaning it cannot be started again. */ - @GuardedBy("lock") - private volatile boolean shutdown; - - // ------------------------------------------------------------------------ - - /** - * Creates a new leader election service. The service assigns the given leader ID - * to the leader contender. - * - * @param leaderId The constant leader ID assigned to the leader. - */ - public SingleLeaderElectionService(Executor notificationsDispatcher, UUID leaderId) { - this.notificationExecutor = checkNotNull(notificationsDispatcher); - this.leaderId = checkNotNull(leaderId); - this.listeners = new HashSet<>(); - - shutdown = false; - } - - // ------------------------------------------------------------------------ - // leader election service - // ------------------------------------------------------------------------ - - @Override - public void start(LeaderContender contender) throws Exception { - checkNotNull(contender, "contender"); - - synchronized (lock) { - checkState(!shutdown, "service is shut down"); - checkState(proposedLeader == null, "service already started"); - - // directly grant leadership to the given contender - proposedLeader = contender; - notificationExecutor.execute(new GrantLeadershipCall(contender, leaderId)); - } - } - - @Override - public void stop() { - synchronized (lock) { - // notify all listeners that there is no leader - for (EmbeddedLeaderRetrievalService listener : listeners) { - notificationExecutor.execute( - new NotifyOfLeaderCall(null, null, listener.listener, LOG)); - } - - // if there was a leader, revoke its leadership - if (leader != null) { - try { - leader.revokeLeadership(); - } catch (Throwable t) { - leader.handleError(t instanceof Exception ? (Exception) t : new Exception(t)); - } - } - - proposedLeader = null; - leader = null; - leaderAddress = null; - } - } - - @Override - public void confirmLeadership(UUID leaderSessionID, String leaderAddress) { - checkNotNull(leaderSessionID, "leaderSessionID"); - checkArgument(leaderSessionID.equals(leaderId), "confirmed wrong leader session id"); - - synchronized (lock) { - checkState(!shutdown, "service is shut down"); - checkState(proposedLeader != null, "no leader proposed yet"); - checkState(leader == null, "leader already confirmed"); - - // accept the confirmation - this.leaderAddress = leaderAddress; - leader = proposedLeader; - - // notify all listeners - for (EmbeddedLeaderRetrievalService listener : listeners) { - notificationExecutor.execute( - new NotifyOfLeaderCall(leaderAddress, leaderId, listener.listener, LOG)); - } - } - } - - @Override - public boolean hasLeadership(@Nonnull UUID leaderSessionId) { - synchronized (lock) { - return proposedLeader != null && leaderSessionId.equals(leaderId); - } - } - - void errorOnGrantLeadership(LeaderContender contender, Throwable error) { - LOG.warn("Error granting leadership to contender", error); - contender.handleError(error instanceof Exception ? (Exception) error : new Exception(error)); - - synchronized (lock) { - if (proposedLeader == contender) { - proposedLeader = null; - leader = null; - } - } - } - - // ------------------------------------------------------------------------ - // shutdown - // ------------------------------------------------------------------------ - - public boolean isShutdown() { - synchronized (lock) { - return shutdown; - } - } - - public void shutdown() { - shutdownInternally(new Exception("The leader service is shutting down")); - } - - private void shutdownInternally(Exception exceptionForHandlers) { - synchronized (lock) { - if (shutdown) { - return; - } - - shutdown = true; - - // fail the leader (if there is one) - if (leader != null) { - try { - leader.handleError(exceptionForHandlers); - } catch (Throwable ignored) {} - } - - // clear all leader status - leader = null; - proposedLeader = null; - leaderAddress = null; - - // fail all registered listeners - for (EmbeddedLeaderRetrievalService service : listeners) { - service.shutdown(exceptionForHandlers); - } - listeners.clear(); - } - } - - private void fatalError(Throwable error) { - LOG.error("Embedded leader election service encountered a fatal error. Shutting down service.", error); - - shutdownInternally(new Exception("Leader election service is shutting down after a fatal error", error)); - } - - // ------------------------------------------------------------------------ - // leader listeners - // ------------------------------------------------------------------------ - - public LeaderRetrievalService createLeaderRetrievalService() { - synchronized (lock) { - checkState(!shutdown, "leader election service is shut down"); - return new EmbeddedLeaderRetrievalService(); - } - } - - void addListener(EmbeddedLeaderRetrievalService service, LeaderRetrievalListener listener) { - synchronized (lock) { - checkState(!shutdown, "leader election service is shut down"); - checkState(!service.running, "leader retrieval service is already started"); - - try { - if (!listeners.add(service)) { - throw new IllegalStateException("leader retrieval service was added to this service multiple times"); - } - - service.listener = listener; - service.running = true; - - // if we already have a leader, immediately notify this new listener - if (leader != null) { - notificationExecutor.execute( - new NotifyOfLeaderCall(leaderAddress, leaderId, listener, LOG)); - } - } - catch (Throwable t) { - fatalError(t); - } - } - } - - void removeListener(EmbeddedLeaderRetrievalService service) { - synchronized (lock) { - // if the service was not even started, simply do nothing - if (!service.running || shutdown) { - return; - } - - try { - if (!listeners.remove(service)) { - throw new IllegalStateException("leader retrieval service does not belong to this service"); - } - - // stop the service - service.listener = null; - service.running = false; - } - catch (Throwable t) { - fatalError(t); - } - } - } - - // ------------------------------------------------------------------------ - - private class EmbeddedLeaderRetrievalService implements LeaderRetrievalService { - - volatile LeaderRetrievalListener listener; - - volatile boolean running; - - @Override - public void start(LeaderRetrievalListener listener) throws Exception { - checkNotNull(listener); - addListener(this, listener); - } - - @Override - public void stop() throws Exception { - removeListener(this); - } - - void shutdown(Exception cause) { - if (running) { - final LeaderRetrievalListener lst = listener; - running = false; - listener = null; - - try { - lst.handleError(cause); - } catch (Throwable ignored) {} - } - } - } - - // ------------------------------------------------------------------------ - // asynchronous notifications - // ------------------------------------------------------------------------ - - /** - * This runnable informs a leader contender that it gained leadership. - */ - private class GrantLeadershipCall implements Runnable { - - private final LeaderContender contender; - private final UUID leaderSessionId; - - GrantLeadershipCall(LeaderContender contender, UUID leaderSessionId) { - - this.contender = checkNotNull(contender); - this.leaderSessionId = checkNotNull(leaderSessionId); - } - - @Override - public void run() { - try { - contender.grantLeadership(leaderSessionId); - } - catch (Throwable t) { - errorOnGrantLeadership(contender, t); - } - } - } - - // ------------------------------------------------------------------------ - - /** - * This runnable informs a leader listener of a new leader. - */ - private static class NotifyOfLeaderCall implements Runnable { - - @Nullable - private final String address; // null if leader revoked without new leader - @Nullable - private final UUID leaderSessionId; // null if leader revoked without new leader - - private final LeaderRetrievalListener listener; - private final Logger logger; - - NotifyOfLeaderCall( - @Nullable String address, - @Nullable UUID leaderSessionId, - LeaderRetrievalListener listener, - Logger logger) { - - this.address = address; - this.leaderSessionId = leaderSessionId; - this.listener = checkNotNull(listener); - this.logger = checkNotNull(logger); - } - - @Override - public void run() { - try { - listener.notifyLeaderAddress(address, leaderSessionId); - } - catch (Throwable t) { - logger.warn("Error notifying leader listener about new leader", t); - listener.handleError(t instanceof Exception ? (Exception) t : new Exception(t)); - } - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java deleted file mode 100644 index b0c7778a2b..0000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FsNegativeRunningJobsRegistryTest.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.highavailability; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.core.fs.Path; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus; -import org.apache.flink.util.TestLogger; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.File; - -import static org.junit.Assert.assertEquals; - -/** - * Tests for the {@link FsNegativeRunningJobsRegistry} on a local file system. - */ -public class FsNegativeRunningJobsRegistryTest extends TestLogger { - - @Rule - public final TemporaryFolder tempFolder = new TemporaryFolder(); - - @Test - public void testCreateAndSetFinished() throws Exception { - final File folder = tempFolder.newFolder(); - final String uri = folder.toURI().toString(); - - final JobID jid = new JobID(); - - FsNegativeRunningJobsRegistry registry = new FsNegativeRunningJobsRegistry(new Path(uri)); - - // another registry should pick this up - FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(new Path(uri)); - - // initially, without any call, the job is pending - assertEquals(JobSchedulingStatus.PENDING, registry.getJobSchedulingStatus(jid)); - assertEquals(JobSchedulingStatus.PENDING, otherRegistry.getJobSchedulingStatus(jid)); - - // after set running, the job is running - registry.setJobRunning(jid); - assertEquals(JobSchedulingStatus.RUNNING, registry.getJobSchedulingStatus(jid)); - assertEquals(JobSchedulingStatus.RUNNING, otherRegistry.getJobSchedulingStatus(jid)); - - // set the job to finished and validate - registry.setJobFinished(jid); - assertEquals(JobSchedulingStatus.DONE, registry.getJobSchedulingStatus(jid)); - assertEquals(JobSchedulingStatus.DONE, otherRegistry.getJobSchedulingStatus(jid)); - } - - @Test - public void testSetFinishedAndRunning() throws Exception { - final File folder = tempFolder.newFolder(); - final String uri = folder.toURI().toString(); - - final JobID jid = new JobID(); - - FsNegativeRunningJobsRegistry registry = new FsNegativeRunningJobsRegistry(new Path(uri)); - - // set the job to finished and validate - registry.setJobFinished(jid); - assertEquals(JobSchedulingStatus.DONE, registry.getJobSchedulingStatus(jid)); - - // set the job to running does not overwrite the finished status - registry.setJobRunning(jid); - assertEquals(JobSchedulingStatus.DONE, registry.getJobSchedulingStatus(jid)); - - // another registry should pick this up - FsNegativeRunningJobsRegistry otherRegistry = new FsNegativeRunningJobsRegistry(new Path(uri)); - assertEquals(JobSchedulingStatus.DONE, otherRegistry.getJobSchedulingStatus(jid)); - - // clear the running and finished marker, it will be pending - otherRegistry.clearJob(jid); - assertEquals(JobSchedulingStatus.PENDING, registry.getJobSchedulingStatus(jid)); - assertEquals(JobSchedulingStatus.PENDING, otherRegistry.getJobSchedulingStatus(jid)); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java deleted file mode 100644 index 910db15999..0000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/leaderelection/SingleLeaderElectionServiceTest.java +++ /dev/null @@ -1,225 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.highavailability.nonha.leaderelection; - -import org.apache.flink.runtime.concurrent.Executors; -import org.apache.flink.runtime.leaderelection.LeaderContender; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.util.StringUtils; - -import org.junit.Test; -import org.mockito.stubbing.Answer; - -import java.util.Random; -import java.util.UUID; -import java.util.concurrent.Executor; - -import static org.junit.Assert.fail; -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.verifyNoMoreInteractions; - -/** - * Tests for the {@link SingleLeaderElectionService}. - */ -public class SingleLeaderElectionServiceTest { - - private static final Random RND = new Random(); - - private final Executor executor = Executors.directExecutor(); - - // ------------------------------------------------------------------------ - - @Test - public void testStartStopAssignLeadership() throws Exception { - final UUID uuid = UUID.randomUUID(); - final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid); - - final LeaderContender contender = mockContender(service); - final LeaderContender otherContender = mockContender(service); - - service.start(contender); - verify(contender, times(1)).grantLeadership(uuid); - - service.stop(); - verify(contender, times(1)).revokeLeadership(); - - // start with a new contender - the old contender must not gain another leadership - service.start(otherContender); - verify(otherContender, times(1)).grantLeadership(uuid); - - verify(contender, times(1)).grantLeadership(uuid); - verify(contender, times(1)).revokeLeadership(); - } - - @Test - public void testStopBeforeConfirmingLeadership() throws Exception { - final UUID uuid = UUID.randomUUID(); - final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid); - - final LeaderContender contender = mock(LeaderContender.class); - - service.start(contender); - verify(contender, times(1)).grantLeadership(uuid); - - service.stop(); - - // because the leadership was never confirmed, there is no "revoke" call - verifyNoMoreInteractions(contender); - } - - @Test - public void testStartOnlyOnce() throws Exception { - final UUID uuid = UUID.randomUUID(); - final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid); - - final LeaderContender contender = mock(LeaderContender.class); - final LeaderContender otherContender = mock(LeaderContender.class); - - service.start(contender); - verify(contender, times(1)).grantLeadership(uuid); - - // should not be possible to start again this with another contender - try { - service.start(otherContender); - fail("should fail with an exception"); - } catch (IllegalStateException e) { - // expected - } - - // should not be possible to start this again with the same contender - try { - service.start(contender); - fail("should fail with an exception"); - } catch (IllegalStateException e) { - // expected - } - } - - @Test - public void testShutdown() throws Exception { - final UUID uuid = UUID.randomUUID(); - final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid); - - // create a leader contender and let it grab leadership - final LeaderContender contender = mockContender(service); - service.start(contender); - verify(contender, times(1)).grantLeadership(uuid); - - // some leader listeners - final LeaderRetrievalListener listener1 = mock(LeaderRetrievalListener.class); - final LeaderRetrievalListener listener2 = mock(LeaderRetrievalListener.class); - - LeaderRetrievalService listenerService1 = service.createLeaderRetrievalService(); - LeaderRetrievalService listenerService2 = service.createLeaderRetrievalService(); - - listenerService1.start(listener1); - listenerService2.start(listener2); - - // one listener stops - listenerService1.stop(); - - // shut down the service - service.shutdown(); - - // the leader contender and running listener should get error notifications - verify(contender, times(1)).handleError(any(Exception.class)); - verify(listener2, times(1)).handleError(any(Exception.class)); - - // the stopped listener gets no notification - verify(listener1, times(0)).handleError(any(Exception.class)); - - // should not be possible to start again after shutdown - try { - service.start(contender); - fail("should fail with an exception"); - } catch (IllegalStateException e) { - // expected - } - - // no additional leadership grant - verify(contender, times(1)).grantLeadership(any(UUID.class)); - } - - @Test - public void testImmediateShutdown() throws Exception { - final UUID uuid = UUID.randomUUID(); - final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid); - service.shutdown(); - - final LeaderContender contender = mock(LeaderContender.class); - - // should not be possible to start - try { - service.start(contender); - fail("should fail with an exception"); - } catch (IllegalStateException e) { - // expected - } - - // no additional leadership grant - verify(contender, times(0)).grantLeadership(any(UUID.class)); - } - -// @Test -// public void testNotifyListenersWhenLeaderElected() throws Exception { -// final UUID uuid = UUID.randomUUID(); -// final SingleLeaderElectionService service = new SingleLeaderElectionService(executor, uuid); -// -// final LeaderRetrievalListener listener1 = mock(LeaderRetrievalListener.class); -// final LeaderRetrievalListener listener2 = mock(LeaderRetrievalListener.class); -// -// LeaderRetrievalService listenerService1 = service.createLeaderRetrievalService(); -// LeaderRetrievalService listenerService2 = service.createLeaderRetrievalService(); -// -// listenerService1.start(listener1); -// listenerService1.start(listener2); -// -// final LeaderContender contender = mockContender(service); -// service.start(contender); -// -// veri -// } - - // ------------------------------------------------------------------------ - // utilities - // ------------------------------------------------------------------------ - - private static LeaderContender mockContender(final LeaderElectionService service) { - String address = StringUtils.getRandomString(RND, 5, 10, 'a', 'z'); - return mockContender(service, address); - } - - private static LeaderContender mockContender(final LeaderElectionService service, final String address) { - LeaderContender mockContender = mock(LeaderContender.class); - - doAnswer((Answer) invocation -> { - final UUID uuid = (UUID) invocation.getArguments()[0]; - service.confirmLeadership(uuid, address); - return null; - }).when(mockContender).grantLeadership(any(UUID.class)); - - return mockContender; - } -} diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/AbstractYarnNonHaServices.java b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/AbstractYarnNonHaServices.java deleted file mode 100644 index cf0b2f945f..0000000000 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/AbstractYarnNonHaServices.java +++ /dev/null @@ -1,93 +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.yarn.highavailability; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; -import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; -import org.apache.flink.runtime.highavailability.FsNegativeRunningJobsRegistry; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; -import org.apache.flink.runtime.jobmanager.JobGraphStore; - -import java.io.IOException; - -/** - * Abstract base class for the high availability services for Flink YARN applications that support - * no master fail over. - * - *

Internally, these services put their recovery data into YARN's working directory, - * except for checkpoints, which are in the configured checkpoint directory. That way, - * checkpoints can be resumed with a new job/application, even if the complete YARN application - * is killed and cleaned up. - */ -public abstract class AbstractYarnNonHaServices extends YarnHighAvailabilityServices { - - // ------------------------------------------------------------------------ - - /** - * Creates new YARN high-availability services, configuring the file system and recovery - * data directory based on the working directory in the given Hadoop configuration. - * - *

This class requires that the default Hadoop file system configured in the given - * Hadoop configuration is an HDFS. - * - * @param config The Flink configuration of this component / process. - * @param hadoopConf The Hadoop configuration for the YARN cluster. - * - * @throws IOException Thrown, if the initialization of the Hadoop file system used by YARN fails. - */ - protected AbstractYarnNonHaServices( - Configuration config, - org.apache.hadoop.conf.Configuration hadoopConf) throws IOException { - super(config, hadoopConf); - } - - // ------------------------------------------------------------------------ - // Services - // ------------------------------------------------------------------------ - - @Override - public RunningJobsRegistry getRunningJobsRegistry() throws IOException { - enter(); - try { - // IMPORTANT: The registry must NOT place its data in a directory that is - // cleaned up by these services. - return new FsNegativeRunningJobsRegistry(flinkFileSystem, workingDirectory); - } - finally { - exit(); - } - } - - @Override - public CheckpointRecoveryFactory getCheckpointRecoveryFactory() { - enter(); - try { - return new StandaloneCheckpointRecoveryFactory(); - } - finally { - exit(); - } - } - - @Override - public JobGraphStore getJobGraphStore() throws Exception { - throw new UnsupportedOperationException("These High-Availability Services do not support storing job graphs"); - } -} diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnHighAvailabilityServices.java b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnHighAvailabilityServices.java deleted file mode 100644 index 0cf9dc483c..0000000000 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnHighAvailabilityServices.java +++ /dev/null @@ -1,371 +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.yarn.highavailability; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.IllegalConfigurationException; -import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.core.fs.Path; -import org.apache.flink.runtime.blob.BlobStore; -import org.apache.flink.runtime.blob.BlobStoreService; -import org.apache.flink.runtime.blob.FileSystemBlobStore; -import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; -import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.InstantiationUtil; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.net.URI; -import java.util.concurrent.locks.ReentrantLock; - -import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed; -import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; - -/** - * The basis of {@link HighAvailabilityServices} for YARN setups. - * These high-availability services auto-configure YARN's HDFS and the YARN application's - * working directory to be used to store job recovery data. - * - *

Note for implementers: This class locks access to and creation of services, - * to make sure all services are properly shut down when shutting down this class. - * To participate in the checks, overriding methods should frame method body with - * calls to {@code enter()} and {@code exit()} as shown in the following pattern: - * - *

{@code
- * public LeaderRetrievalService getResourceManagerLeaderRetriever() {
- *     enter();
- *     try {
- *         CuratorClient client = getCuratorClient();
- *         return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
- *     } finally {
- *         exit();
- *     }
- * }
- * }
- */ -public abstract class YarnHighAvailabilityServices implements HighAvailabilityServices { - - /** The name of the sub directory in which Flink stores the recovery data. */ - public static final String FLINK_RECOVERY_DATA_DIR = "flink_recovery_data"; - - /** Logger for these services, shared with subclasses. */ - protected static final Logger LOG = LoggerFactory.getLogger(YarnHighAvailabilityServices.class); - - // ------------------------------------------------------------------------ - - /** The lock that guards all accesses to methods in this class. */ - private final ReentrantLock lock; - - /** The Flink FileSystem object that represent the HDFS used by YARN. */ - protected final FileSystem flinkFileSystem; - - /** The Hadoop FileSystem object that represent the HDFS used by YARN. */ - protected final org.apache.hadoop.fs.FileSystem hadoopFileSystem; - - /** The working directory of this YARN application. - * This MUST NOT be deleted when the HA services clean up */ - protected final Path workingDirectory; - - /** The directory for HA persistent data. This should be deleted when the - * HA services clean up. */ - protected final Path haDataDirectory; - - /** Blob store service to be used for the BlobServer and BlobCache. */ - protected final BlobStoreService blobStoreService; - - /** Flag marking this instance as shut down. */ - private volatile boolean closed; - - // ------------------------------------------------------------------------ - - /** - * Creates new YARN high-availability services, configuring the file system and recovery - * data directory based on the working directory in the given Hadoop configuration. - * - *

This class requires that the default Hadoop file system configured in the given - * Hadoop configuration is an HDFS. - * - * @param config The Flink configuration of this component / process. - * @param hadoopConf The Hadoop configuration for the YARN cluster. - * - * @throws IOException Thrown, if the initialization of the Hadoop file system used by YARN fails. - */ - protected YarnHighAvailabilityServices( - Configuration config, - org.apache.hadoop.conf.Configuration hadoopConf) throws IOException { - - checkNotNull(config); - checkNotNull(hadoopConf); - - this.lock = new ReentrantLock(); - - // get and verify the YARN HDFS URI - final URI fsUri = org.apache.hadoop.fs.FileSystem.getDefaultUri(hadoopConf); - if (fsUri.getScheme() == null || !"hdfs".equals(fsUri.getScheme().toLowerCase())) { - throw new IOException("Invalid file system found for YarnHighAvailabilityServices: " + - "Expected 'hdfs', but found '" + fsUri.getScheme() + "'."); - } - - // initialize the Hadoop File System - // we go through this special code path here to make sure we get no shared cached - // instance of the FileSystem - try { - final Class fsClass = - org.apache.hadoop.fs.FileSystem.getFileSystemClass(fsUri.getScheme(), hadoopConf); - - this.hadoopFileSystem = InstantiationUtil.instantiate(fsClass); - this.hadoopFileSystem.initialize(fsUri, hadoopConf); - } - catch (Exception e) { - throw new IOException("Cannot instantiate YARN's Hadoop file system for " + fsUri, e); - } - - this.flinkFileSystem = new HadoopFileSystem(hadoopFileSystem); - - this.workingDirectory = new Path(hadoopFileSystem.getWorkingDirectory().toUri()); - this.haDataDirectory = new Path(workingDirectory, FLINK_RECOVERY_DATA_DIR); - - // test the file system, to make sure we fail fast if access does not work - try { - flinkFileSystem.mkdirs(haDataDirectory); - } - catch (Exception e) { - throw new IOException("Could not create the directory for recovery data in YARN's file system at '" - + haDataDirectory + "'.", e); - } - - LOG.info("Flink YARN application will store recovery data at {}", haDataDirectory); - - blobStoreService = new FileSystemBlobStore(flinkFileSystem, haDataDirectory.toString()); - } - - // ------------------------------------------------------------------------ - // high availability services - // ------------------------------------------------------------------------ - - @Override - public BlobStore createBlobStore() throws IOException { - enter(); - try { - return blobStoreService; - } finally { - exit(); - } - } - - // ------------------------------------------------------------------------ - // Shutdown - // ------------------------------------------------------------------------ - - /** - * Checks whether these services have been shut down. - * - * @return True, if this instance has been shut down, false if it still operational. - */ - public boolean isClosed() { - return closed; - } - - @Override - public void close() throws Exception { - lock.lock(); - try { - // close only once - if (closed) { - return; - } - closed = true; - - Throwable exception = null; - - try { - blobStoreService.close(); - } catch (Throwable t) { - exception = t; - } - - // we do not propagate exceptions here, but only log them - try { - hadoopFileSystem.close(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - - if (exception != null) { - ExceptionUtils.rethrowException(exception, "Could not properly close the YarnHighAvailabilityServices."); - } - } - finally { - lock.unlock(); - } - } - - @Override - public void closeAndCleanupAllData() throws Exception { - lock.lock(); - try { - checkState(!closed, "YarnHighAvailabilityServices are already closed"); - - // we remember exceptions only, then continue cleanup, and re-throw at the end - Throwable exception = null; - - try { - blobStoreService.closeAndCleanupAllData(); - } catch (Throwable t) { - exception = t; - } - - // first, we delete all data in Flink's data directory - try { - flinkFileSystem.delete(haDataDirectory, true); - } - catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - - // now we actually close the services - try { - close(); - } - catch (Throwable t) { - exception = firstOrSuppressed(t, exception); - } - - // if some exception occurred, rethrow - if (exception != null) { - ExceptionUtils.rethrowException(exception, exception.getMessage()); - } - } - finally { - lock.unlock(); - } - } - - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - /** - * To be called at the beginning of every method that creates an HA service. Acquires the lock - * and check whether this HighAvailabilityServices instance is shut down. - */ - void enter() { - if (!enterUnlessClosed()) { - throw new IllegalStateException("closed"); - } - } - - /** - * Acquires the lock and checks whether the services are already closed. If they are - * already closed, the method releases the lock and returns {@code false}. - * - * @return True, if the lock was acquired and the services are not closed, false if the services are closed. - */ - boolean enterUnlessClosed() { - lock.lock(); - if (!closed) { - return true; - } else { - lock.unlock(); - return false; - } - } - - /** - * To be called at the end of every method that creates an HA service. Releases the lock. - */ - void exit() { - lock.unlock(); - } - - // ------------------------------------------------------------------------ - // Factory from Configuration - // ------------------------------------------------------------------------ - - /** - * Creates the high-availability services for a single-job Flink YARN application, to be - * used in the Application Master that runs both ResourceManager and JobManager. - * - * @param flinkConfig The Flink configuration. - * @param hadoopConfig The Hadoop configuration for the YARN cluster. - * - * @return The created high-availability services. - * - * @throws IOException Thrown, if the high-availability services could not be initialized. - */ - public static YarnHighAvailabilityServices forSingleJobAppMaster( - Configuration flinkConfig, - org.apache.hadoop.conf.Configuration hadoopConfig) throws IOException { - - checkNotNull(flinkConfig, "flinkConfig"); - checkNotNull(hadoopConfig, "hadoopConfig"); - - final HighAvailabilityMode mode = HighAvailabilityMode.fromConfig(flinkConfig); - switch (mode) { - case NONE: - return new YarnIntraNonHaMasterServices(flinkConfig, hadoopConfig); - - case ZOOKEEPER: - throw new UnsupportedOperationException("to be implemented"); - - default: - throw new IllegalConfigurationException("Unrecognized high availability mode: " + mode); - } - } - - /** - * Creates the high-availability services for the TaskManagers participating in - * a Flink YARN application. - * - * @param flinkConfig The Flink configuration. - * @param hadoopConfig The Hadoop configuration for the YARN cluster. - * - * @return The created high-availability services. - * - * @throws IOException Thrown, if the high-availability services could not be initialized. - */ - public static YarnHighAvailabilityServices forYarnTaskManager( - Configuration flinkConfig, - org.apache.hadoop.conf.Configuration hadoopConfig) throws IOException { - - checkNotNull(flinkConfig, "flinkConfig"); - checkNotNull(hadoopConfig, "hadoopConfig"); - - final HighAvailabilityMode mode = HighAvailabilityMode.fromConfig(flinkConfig); - switch (mode) { - case NONE: - return new YarnPreConfiguredMasterNonHaServices( - flinkConfig, - hadoopConfig, - HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION); - - case ZOOKEEPER: - throw new UnsupportedOperationException("to be implemented"); - - default: - throw new IllegalConfigurationException("Unrecognized high availability mode: " + mode); - } - } -} diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java deleted file mode 100644 index 466e198dda..0000000000 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java +++ /dev/null @@ -1,246 +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.yarn.highavailability; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.IllegalConfigurationException; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.ServicesThreadFactory; -import org.apache.flink.runtime.highavailability.nonha.leaderelection.SingleLeaderElectionService; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; - -import java.io.IOException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -/** - * These YarnHighAvailabilityServices are for the Application Master in setups where there is one - * ResourceManager that is statically configured in the Flink configuration. - * - *

Handled failure types

- *
    - *
  • User code & operator failures: Failed operators are recovered from checkpoints.
  • - *
  • Task Manager Failures: Failed Task Managers are restarted and their tasks are - * recovered from checkpoints.
  • - *
- * - *

Non-recoverable failure types

- *
    - *
  • Application Master failures: These failures cannot be recovered, because TaskManagers - * have no way to discover the new Application Master's address.
  • - *
- * - *

Internally, these services put their recovery data into YARN's working directory, - * except for checkpoints, which are in the configured checkpoint directory. That way, - * checkpoints can be resumed with a new job/application, even if the complete YARN application - * is killed and cleaned up. - * - *

Because ResourceManager and JobManager run both in the same process (Application Master), they - * use an embedded leader election service to find each other. - * - *

A typical YARN setup that uses these HA services first starts the ResourceManager - * inside the ApplicationMaster and puts its RPC endpoint address into the configuration with which - * the TaskManagers are started. Because of this static addressing scheme, the setup cannot handle failures - * of the JobManager and ResourceManager, which are running as part of the Application Master. - * - * @see HighAvailabilityServices - */ -public class YarnIntraNonHaMasterServices extends AbstractYarnNonHaServices { - - /** The dispatcher thread pool for these services. */ - private final ExecutorService dispatcher; - - /** The embedded leader election service used by JobManagers to find the resource manager. */ - private final SingleLeaderElectionService resourceManagerLeaderElectionService; - - /** The embedded leader election service for the dispatcher. */ - private final SingleLeaderElectionService dispatcherLeaderElectionService; - - // ------------------------------------------------------------------------ - - /** - * Creates new YarnIntraNonHaMasterServices for the given Flink and YARN configuration. - * - *

This constructor initializes access to the HDFS to store recovery data, and creates the - * embedded leader election services through which ResourceManager and JobManager find and - * confirm each other. - * - * @param config The Flink configuration of this component / process. - * @param hadoopConf The Hadoop configuration for the YARN cluster. - * - * @throws IOException - * Thrown, if the initialization of the Hadoop file system used by YARN fails. - * @throws IllegalConfigurationException - * Thrown, if the Flink configuration does not properly describe the ResourceManager address and port. - */ - public YarnIntraNonHaMasterServices( - Configuration config, - org.apache.hadoop.conf.Configuration hadoopConf) throws IOException { - - super(config, hadoopConf); - - // track whether we successfully perform the initialization - boolean successful = false; - - try { - this.dispatcher = Executors.newSingleThreadExecutor(new ServicesThreadFactory()); - this.resourceManagerLeaderElectionService = new SingleLeaderElectionService(dispatcher, DEFAULT_LEADER_ID); - this.dispatcherLeaderElectionService = new SingleLeaderElectionService(dispatcher, DEFAULT_LEADER_ID); - - // all good! - successful = true; - } - finally { - if (!successful) { - // quietly undo what the parent constructor initialized - try { - super.close(); - } catch (Throwable ignored) {} - } - } - } - - // ------------------------------------------------------------------------ - // Services - // ------------------------------------------------------------------------ - - @Override - public LeaderRetrievalService getResourceManagerLeaderRetriever() { - enter(); - try { - return resourceManagerLeaderElectionService.createLeaderRetrievalService(); - } - finally { - exit(); - } - } - - @Override - public LeaderRetrievalService getDispatcherLeaderRetriever() { - enter(); - - try { - return dispatcherLeaderElectionService.createLeaderRetrievalService(); - } finally { - exit(); - } - } - - @Override - public LeaderElectionService getResourceManagerLeaderElectionService() { - enter(); - try { - return resourceManagerLeaderElectionService; - } - finally { - exit(); - } - } - - @Override - public LeaderElectionService getDispatcherLeaderElectionService() { - enter(); - try { - return dispatcherLeaderElectionService; - } finally { - exit(); - } - } - - @Override - public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) { - enter(); - try { - throw new UnsupportedOperationException("needs refactoring to accept default address"); - } - finally { - exit(); - } - } - - @Override - public LeaderElectionService getClusterRestEndpointLeaderElectionService() { - enter(); - try { - throw new UnsupportedOperationException(); - } - finally { - exit(); - } - } - - @Override - public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) { - enter(); - try { - throw new UnsupportedOperationException("needs refactoring to accept default address"); - } - finally { - exit(); - } - } - - @Override - public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultJobManagerAddress) { - enter(); - try { - throw new UnsupportedOperationException("needs refactoring to accept default address"); - } - finally { - exit(); - } - } - - @Override - public LeaderRetrievalService getClusterRestEndpointLeaderRetriever() { - enter(); - try { - throw new UnsupportedOperationException(); - } - finally { - exit(); - } - } - - // ------------------------------------------------------------------------ - // shutdown - // ------------------------------------------------------------------------ - - @Override - public void close() throws Exception { - if (enterUnlessClosed()) { - try { - try { - // this class' own cleanup logic - resourceManagerLeaderElectionService.shutdown(); - dispatcher.shutdownNow(); - } - finally { - // in any case must we call the parent cleanup logic - super.close(); - } - } - finally { - exit(); - } - } - } -} diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java deleted file mode 100644 index f80c6b17c7..0000000000 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java +++ /dev/null @@ -1,243 +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.yarn.highavailability; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.IllegalConfigurationException; -import org.apache.flink.runtime.dispatcher.Dispatcher; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService; -import org.apache.flink.runtime.resourcemanager.ResourceManager; -import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; -import org.apache.flink.yarn.configuration.YarnConfigOptions; - -import java.io.IOException; - -/** - * These YarnHighAvailabilityServices are for use by the TaskManager in setups, - * where there is one ResourceManager that is statically configured in the Flink configuration. - * - *

Handled failure types

- *
    - *
  • User code & operator failures: Failed operators are recovered from checkpoints.
  • - *
  • Task Manager Failures: Failed Task Managers are restarted and their tasks are - * recovered from checkpoints.
  • - *
- * - *

Non-recoverable failure types

- *
    - *
  • Application Master failures: These failures cannot be recovered, because TaskManagers - * have no way to discover the new Application Master's address.
  • - *
- * - *

Internally, these services put their recovery data into YARN's working directory, - * except for checkpoints, which are in the configured checkpoint directory. That way, - * checkpoints can be resumed with a new job/application, even if the complete YARN application - * is killed and cleaned up. - * - *

A typical YARN setup that uses these HA services first starts the ResourceManager - * inside the ApplicationMaster and puts its RPC endpoint address into the configuration with which - * the TaskManagers are started. Because of this static addressing scheme, the setup cannot handle failures - * of the JobManager and ResourceManager, which are running as part of the Application Master. - * - * @see HighAvailabilityServices - */ -public class YarnPreConfiguredMasterNonHaServices extends AbstractYarnNonHaServices { - - /** The RPC URL under which the single ResourceManager can be reached while available. */ - private final String resourceManagerRpcUrl; - - /** The RPC URL under which the single Dispatcher can be reached while available. */ - private final String dispatcherRpcUrl; - - // ------------------------------------------------------------------------ - - /** - * Creates new YarnPreConfiguredMasterHaServices for the given Flink and YARN configuration. - * This constructor parses the ResourceManager address from the Flink configuration and sets - * up the HDFS access to store recovery data in the YARN application's working directory. - * - * @param config The Flink configuration of this component / process. - * @param hadoopConf The Hadoop configuration for the YARN cluster. - * - * @throws IOException - * Thrown, if the initialization of the Hadoop file system used by YARN fails. - * @throws IllegalConfigurationException - * Thrown, if the Flink configuration does not properly describe the ResourceManager address and port. - */ - public YarnPreConfiguredMasterNonHaServices( - Configuration config, - org.apache.hadoop.conf.Configuration hadoopConf, - HighAvailabilityServicesUtils.AddressResolution addressResolution) throws IOException { - - super(config, hadoopConf); - - // track whether we successfully perform the initialization - boolean successful = false; - - try { - // extract the hostname and port of the resource manager - final String rmHost = config.getString(YarnConfigOptions.APP_MASTER_RPC_ADDRESS); - final int rmPort = config.getInteger(YarnConfigOptions.APP_MASTER_RPC_PORT); - - if (rmHost == null) { - throw new IllegalConfigurationException("Config parameter '" + - YarnConfigOptions.APP_MASTER_RPC_ADDRESS.key() + "' is missing."); - } - if (rmPort < 0) { - throw new IllegalConfigurationException("Config parameter '" + - YarnConfigOptions.APP_MASTER_RPC_PORT.key() + "' is missing."); - } - if (rmPort <= 0 || rmPort >= 65536) { - throw new IllegalConfigurationException("Invalid value for '" + - YarnConfigOptions.APP_MASTER_RPC_PORT.key() + "' - port must be in [1, 65535]"); - } - - this.resourceManagerRpcUrl = AkkaRpcServiceUtils.getRpcUrl( - rmHost, - rmPort, - ResourceManager.RESOURCE_MANAGER_NAME, - addressResolution, - config); - - this.dispatcherRpcUrl = AkkaRpcServiceUtils.getRpcUrl( - rmHost, - rmPort, - Dispatcher.DISPATCHER_NAME, - addressResolution, - config); - - // all well! - successful = true; - } - finally { - if (!successful) { - // quietly undo what the parent constructor initialized - try { - super.close(); - } catch (Throwable ignored) {} - } - } - } - - // ------------------------------------------------------------------------ - // Services - // ------------------------------------------------------------------------ - - @Override - public LeaderRetrievalService getResourceManagerLeaderRetriever() { - enter(); - try { - return new StandaloneLeaderRetrievalService(resourceManagerRpcUrl, DEFAULT_LEADER_ID); - } - finally { - exit(); - } - } - - @Override - public LeaderRetrievalService getDispatcherLeaderRetriever() { - enter(); - - try { - return new StandaloneLeaderRetrievalService(dispatcherRpcUrl, DEFAULT_LEADER_ID); - } finally { - exit(); - } - } - - @Override - public LeaderElectionService getResourceManagerLeaderElectionService() { - enter(); - try { - throw new UnsupportedOperationException("Not supported on the TaskManager side"); - } - finally { - exit(); - } - } - - @Override - public LeaderElectionService getDispatcherLeaderElectionService() { - enter(); - try { - throw new UnsupportedOperationException("Not supported on the TaskManager side"); - } finally { - exit(); - } - } - - @Override - public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) { - enter(); - try { - throw new UnsupportedOperationException("needs refactoring to accept default address"); - } - finally { - exit(); - } - } - - @Override - public LeaderElectionService getClusterRestEndpointLeaderElectionService() { - enter(); - try { - throw new UnsupportedOperationException(); - } - finally { - exit(); - } - } - - @Override - public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) { - enter(); - try { - throw new UnsupportedOperationException("needs refactoring to accept default address"); - } - finally { - exit(); - } - } - - @Override - public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID, String defaultJobManagerAddress) { - enter(); - try { - return new StandaloneLeaderRetrievalService(defaultJobManagerAddress, DEFAULT_LEADER_ID); - } finally { - exit(); - } - } - - @Override - public LeaderRetrievalService getClusterRestEndpointLeaderRetriever() { - enter(); - try { - throw new UnsupportedOperationException(); - } - finally { - exit(); - } - } -} diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java deleted file mode 100644 index ab4f70a482..0000000000 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java +++ /dev/null @@ -1,163 +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.yarn.highavailability; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.Path; -import org.apache.flink.runtime.leaderelection.LeaderContender; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.util.OperatingSystem; -import org.apache.flink.util.StringUtils; -import org.apache.flink.util.TestLogger; - -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.AfterClass; -import org.junit.Assume; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -import java.io.File; -import java.util.Random; -import java.util.UUID; - -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyString; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.timeout; -import static org.mockito.Mockito.verify; - -/** - * Tests for YarnIntraNonHaMasterServices. - */ -public class YarnIntraNonHaMasterServicesTest extends TestLogger { - - private static final Random RND = new Random(); - - @ClassRule - public static final TemporaryFolder TEMP_DIR = new TemporaryFolder(); - - private static MiniDFSCluster hdfsCluster; - - private static Path hdfsRootPath; - - private org.apache.hadoop.conf.Configuration hadoopConfig; - - // ------------------------------------------------------------------------ - // Test setup and shutdown - // ------------------------------------------------------------------------ - - @BeforeClass - public static void createHDFS() throws Exception { - Assume.assumeTrue(!OperatingSystem.isWindows()); - - final File tempDir = TEMP_DIR.newFolder(); - - org.apache.hadoop.conf.Configuration hdConf = new org.apache.hadoop.conf.Configuration(); - hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, tempDir.getAbsolutePath()); - - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf); - hdfsCluster = builder.build(); - hdfsRootPath = new Path(hdfsCluster.getURI()); - } - - @AfterClass - public static void destroyHDFS() { - if (hdfsCluster != null) { - hdfsCluster.shutdown(); - } - hdfsCluster = null; - hdfsRootPath = null; - } - - @Before - public void initConfig() { - hadoopConfig = new org.apache.hadoop.conf.Configuration(); - hadoopConfig.set(org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY, hdfsRootPath.toString()); - } - - // ------------------------------------------------------------------------ - // Tests - // ------------------------------------------------------------------------ - - @Test - public void testRepeatedClose() throws Exception { - final Configuration flinkConfig = new Configuration(); - - final YarnHighAvailabilityServices services = new YarnIntraNonHaMasterServices(flinkConfig, hadoopConfig); - services.closeAndCleanupAllData(); - - // this should not throw an exception - services.close(); - } - - @Test - public void testClosingReportsToLeader() throws Exception { - final Configuration flinkConfig = new Configuration(); - - try (YarnHighAvailabilityServices services = new YarnIntraNonHaMasterServices(flinkConfig, hadoopConfig)) { - final LeaderElectionService elector = services.getResourceManagerLeaderElectionService(); - final LeaderRetrievalService retrieval = services.getResourceManagerLeaderRetriever(); - final LeaderContender contender = mockContender(elector); - final LeaderRetrievalListener listener = mock(LeaderRetrievalListener.class); - - elector.start(contender); - retrieval.start(listener); - - // wait until the contender has become the leader - verify(listener, timeout(1000L).times(1)).notifyLeaderAddress(anyString(), any(UUID.class)); - - // now we can close the election service - services.close(); - - verify(contender, timeout(1000L).times(1)).handleError(any(Exception.class)); - } - } - - // ------------------------------------------------------------------------ - // utilities - // ------------------------------------------------------------------------ - - private static LeaderContender mockContender(final LeaderElectionService service) { - String address = StringUtils.getRandomString(RND, 5, 10, 'a', 'z'); - return mockContender(service, address); - } - - private static LeaderContender mockContender(final LeaderElectionService service, final String address) { - LeaderContender mockContender = mock(LeaderContender.class); - - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - final UUID uuid = (UUID) invocation.getArguments()[0]; - service.confirmLeadership(uuid, address); - return null; - } - }).when(mockContender).grantLeadership(any(UUID.class)); - - return mockContender; - } -} diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterHaServicesTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterHaServicesTest.java deleted file mode 100644 index 67186e985b..0000000000 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterHaServicesTest.java +++ /dev/null @@ -1,225 +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.yarn.highavailability; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.IllegalConfigurationException; -import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.core.fs.Path; -import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; -import org.apache.flink.util.OperatingSystem; -import org.apache.flink.util.TestLogger; -import org.apache.flink.yarn.configuration.YarnConfigOptions; - -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.AfterClass; -import org.junit.Assume; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.File; -import java.io.FileNotFoundException; - -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** - * Tests for YarnPreConfiguredMasterNonHaServices. - */ -public class YarnPreConfiguredMasterHaServicesTest extends TestLogger { - - @ClassRule - public static final TemporaryFolder TEMP_DIR = new TemporaryFolder(); - - private static MiniDFSCluster hdfsCluster; - - private static Path hdfsRootPath; - - private org.apache.hadoop.conf.Configuration hadoopConfig; - - // ------------------------------------------------------------------------ - // Test setup and shutdown - // ------------------------------------------------------------------------ - - @BeforeClass - public static void createHDFS() throws Exception { - Assume.assumeTrue(!OperatingSystem.isWindows()); - - final File tempDir = TEMP_DIR.newFolder(); - - org.apache.hadoop.conf.Configuration hdConf = new org.apache.hadoop.conf.Configuration(); - hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, tempDir.getAbsolutePath()); - - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf); - hdfsCluster = builder.build(); - hdfsRootPath = new Path(hdfsCluster.getURI()); - } - - @AfterClass - public static void destroyHDFS() { - if (hdfsCluster != null) { - hdfsCluster.shutdown(); - } - hdfsCluster = null; - hdfsRootPath = null; - } - - @Before - public void initConfig() { - hadoopConfig = new org.apache.hadoop.conf.Configuration(); - hadoopConfig.set(org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY, hdfsRootPath.toString()); - } - - // ------------------------------------------------------------------------ - // Tests - // ------------------------------------------------------------------------ - - @Test - public void testMissingRmConfiguration() throws Exception { - final Configuration flinkConfig = new Configuration(); - - // missing resource manager address - try { - new YarnPreConfiguredMasterNonHaServices( - flinkConfig, - hadoopConfig, - HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION); - fail(); - } catch (IllegalConfigurationException e) { - // expected - } - - flinkConfig.setString(YarnConfigOptions.APP_MASTER_RPC_ADDRESS, "localhost"); - - // missing resource manager port - try { - new YarnPreConfiguredMasterNonHaServices( - flinkConfig, - hadoopConfig, - HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION); - fail(); - } catch (IllegalConfigurationException e) { - // expected - } - - flinkConfig.setInteger(YarnConfigOptions.APP_MASTER_RPC_PORT, 1427); - - // now everything is good ;-) - new YarnPreConfiguredMasterNonHaServices( - flinkConfig, - hadoopConfig, - HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION).closeAndCleanupAllData(); - } - - @Test - public void testCloseAndCleanup() throws Exception { - final Configuration flinkConfig = new Configuration(); - flinkConfig.setString(YarnConfigOptions.APP_MASTER_RPC_ADDRESS, "localhost"); - flinkConfig.setInteger(YarnConfigOptions.APP_MASTER_RPC_PORT, 1427); - - // create the services - YarnHighAvailabilityServices services = new YarnPreConfiguredMasterNonHaServices( - flinkConfig, - hadoopConfig, - HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION); - services.closeAndCleanupAllData(); - - final FileSystem fileSystem = hdfsRootPath.getFileSystem(); - final Path workDir = new Path(hdfsCluster.getFileSystem().getWorkingDirectory().toString()); - - try { - fileSystem.getFileStatus(new Path(workDir, YarnHighAvailabilityServices.FLINK_RECOVERY_DATA_DIR)); - fail("Flink recovery data directory still exists"); - } - catch (FileNotFoundException e) { - // expected, because the directory should have been cleaned up - } - - assertTrue(services.isClosed()); - - // doing another cleanup when the services are closed should fail - try { - services.closeAndCleanupAllData(); - fail("should fail with an IllegalStateException"); - } catch (IllegalStateException e) { - // expected - } - } - - @Test - public void testCallsOnClosedServices() throws Exception { - final Configuration flinkConfig = new Configuration(); - flinkConfig.setString(YarnConfigOptions.APP_MASTER_RPC_ADDRESS, "localhost"); - flinkConfig.setInteger(YarnConfigOptions.APP_MASTER_RPC_PORT, 1427); - - YarnHighAvailabilityServices services = new YarnPreConfiguredMasterNonHaServices( - flinkConfig, - hadoopConfig, - HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION); - - // this method is not supported - try { - services.getJobGraphStore(); - fail(); - } catch (UnsupportedOperationException ignored) {} - - services.close(); - - // all these methods should fail now - - try { - services.createBlobStore(); - fail(); - } catch (IllegalStateException ignored) {} - - try { - services.getCheckpointRecoveryFactory(); - fail(); - } catch (IllegalStateException ignored) {} - - try { - services.getJobManagerLeaderElectionService(new JobID()); - fail(); - } catch (IllegalStateException ignored) {} - - try { - services.getJobManagerLeaderRetriever(new JobID()); - fail(); - } catch (IllegalStateException ignored) {} - - try { - services.getRunningJobsRegistry(); - fail(); - } catch (IllegalStateException ignored) {} - - try { - services.getResourceManagerLeaderElectionService(); - fail(); - } catch (IllegalStateException ignored) {} - - try { - services.getResourceManagerLeaderRetriever(); - fail(); - } catch (IllegalStateException ignored) {} - } -} -- Gitee From cee6e4d95af8a8cd4806bd1d8647f1b5674b61be Mon Sep 17 00:00:00 2001 From: whlwanghailong Date: Tue, 1 Oct 2019 06:03:45 +0800 Subject: [PATCH 185/268] [FLINK-14273][table-planner] Improve exception message when signature validation of UDAF is failed This closes #9823 --- .../functions/utils/AggSqlFunction.scala | 25 +++++++++++-------- .../functions/utils/AggSqlFunction.scala | 21 ++++++++++------ .../UserDefinedFunctionValidationTest.scala | 3 ++- 3 files changed, 31 insertions(+), 18 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/AggSqlFunction.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/AggSqlFunction.scala index c6ca9a38f6..7effbba7e5 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/AggSqlFunction.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/AggSqlFunction.scala @@ -61,8 +61,8 @@ class AggSqlFunction( new SqlIdentifier(name, SqlParserPos.ZERO), returnTypeInfer.getOrElse(createReturnTypeInference( fromDataTypeToLogicalType(externalResultType), typeFactory)), - createOperandTypeInference(name, aggregateFunction, typeFactory), - createOperandTypeChecker(name, aggregateFunction), + createOperandTypeInference(name, aggregateFunction, typeFactory, externalAccType), + createOperandTypeChecker(name, aggregateFunction, externalAccType), // Do not need to provide a calcite aggregateFunction here. Flink aggregateion function // will be generated when translating the calcite relnode to flink runtime execution plan null, @@ -107,7 +107,8 @@ object AggSqlFunction { private[flink] def createOperandTypeInference( name: String, aggregateFunction: UserDefinedAggregateFunction[_, _], - typeFactory: FlinkTypeFactory): SqlOperandTypeInference = { + typeFactory: FlinkTypeFactory, + externalAccType: DataType): SqlOperandTypeInference = { /** * Operand type inference based on [[AggregateFunction]] given information. */ @@ -117,13 +118,14 @@ object AggSqlFunction { returnType: RelDataType, operandTypes: Array[RelDataType]): Unit = { - val operandTypeInfo = getOperandType(callBinding) + val operandLogicalType = getOperandType(callBinding) + val actualSignature = externalAccType.getLogicalType +: operandLogicalType - val foundSignature = getAccumulateMethodSignature(aggregateFunction, operandTypeInfo) + val foundSignature = getAccumulateMethodSignature(aggregateFunction, operandLogicalType) .getOrElse( throw new ValidationException( s"Given parameters of function '$name' do not match any signature. \n" + - s"Actual: ${signatureInternalToString(operandTypeInfo)} \n" + + s"Actual: ${signatureInternalToString(actualSignature)} \n" + s"Expected: ${signaturesToString(aggregateFunction, "accumulate")}")) val inferredTypes = getParameterTypes(aggregateFunction, foundSignature.drop(1)) @@ -156,7 +158,8 @@ object AggSqlFunction { private[flink] def createOperandTypeChecker( name: String, - aggregateFunction: UserDefinedAggregateFunction[_, _]): SqlOperandTypeChecker = { + aggregateFunction: UserDefinedAggregateFunction[_, _], + externalAccType: DataType): SqlOperandTypeChecker = { val methods = checkAndExtractMethods(aggregateFunction, "accumulate") @@ -194,15 +197,17 @@ object AggSqlFunction { override def checkOperandTypes( callBinding: SqlCallBinding, throwOnFailure: Boolean): Boolean = { - val operandTypeInfo = getOperandType(callBinding) - val foundSignature = getAccumulateMethodSignature(aggregateFunction, operandTypeInfo) + val operandLogicalType = getOperandType(callBinding) + val actualSignature = externalAccType.getLogicalType +: operandLogicalType + + val foundSignature = getAccumulateMethodSignature(aggregateFunction, operandLogicalType) if (foundSignature.isEmpty) { if (throwOnFailure) { throw new ValidationException( s"Given parameters of function '$name' do not match any signature. \n" + - s"Actual: ${signatureInternalToString(operandTypeInfo)} \n" + + s"Actual: ${signatureInternalToString(actualSignature)} \n" + s"Expected: ${signaturesToString(aggregateFunction, "accumulate")}") } else { false diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala index 4859c2a3d2..de71825dda 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala @@ -56,8 +56,8 @@ class AggSqlFunction( extends SqlUserDefinedAggFunction( new SqlIdentifier(name, SqlParserPos.ZERO), createReturnTypeInference(returnType, typeFactory), - createOperandTypeInference(aggregateFunction, typeFactory), - createOperandTypeChecker(aggregateFunction), + createOperandTypeInference(aggregateFunction, typeFactory, accType), + createOperandTypeChecker(aggregateFunction, accType), // Do not need to provide a calcite aggregateFunction here. Flink aggregation function // will be generated when translating the calcite relnode to flink runtime execution plan null, @@ -104,7 +104,8 @@ object AggSqlFunction { private[flink] def createOperandTypeInference( aggregateFunction: UserDefinedAggregateFunction[_, _], - typeFactory: FlinkTypeFactory) + typeFactory: FlinkTypeFactory, + accType: TypeInformation[_]) : SqlOperandTypeInference = { /** * Operand type inference based on [[AggregateFunction]] given information. @@ -117,11 +118,13 @@ object AggSqlFunction { val operandTypeInfo = getOperandTypeInfo(callBinding) + val actualSignature = accType +: operandTypeInfo + val foundSignature = getAccumulateMethodSignature(aggregateFunction, operandTypeInfo) .getOrElse( throw new ValidationException( s"Given parameters of function do not match any signature. \n" + - s"Actual: ${signatureToString(operandTypeInfo)} \n" + + s"Actual: ${signatureToString(actualSignature)} \n" + s"Expected: ${signaturesToString(aggregateFunction, "accumulate")}")) val inferredTypes = getParameterTypes(aggregateFunction, foundSignature.drop(1)) @@ -153,8 +156,10 @@ object AggSqlFunction { } } - private[flink] def createOperandTypeChecker(aggregateFunction: UserDefinedAggregateFunction[_, _]) - : SqlOperandTypeChecker = { + private[flink] def createOperandTypeChecker( + aggregateFunction: UserDefinedAggregateFunction[_, _], + accType: TypeInformation[_]) + : SqlOperandTypeChecker = { val methods = checkAndExtractMethods(aggregateFunction, "accumulate") @@ -195,13 +200,15 @@ object AggSqlFunction { : Boolean = { val operandTypeInfo = getOperandTypeInfo(callBinding) + val actualSignature = accType +: operandTypeInfo + val foundSignature = getAccumulateMethodSignature(aggregateFunction, operandTypeInfo) if (foundSignature.isEmpty) { if (throwOnFailure) { throw new ValidationException( s"Given parameters of function do not match any signature. \n" + - s"Actual: ${signatureToString(operandTypeInfo)} \n" + + s"Actual: ${signatureToString(actualSignature)} \n" + s"Expected: ${signaturesToString(aggregateFunction, "accumulate")}") } else { false diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala index aeb226eb35..4ba2d5227f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala @@ -45,7 +45,8 @@ class UserDefinedFunctionValidationTest extends TableTestBase { thrown.expect(classOf[ValidationException]) thrown.expectMessage( "Given parameters of function do not match any signature. \n" + - "Actual: (java.lang.String, java.lang.Integer) \n" + + "Actual: (org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions" + + ".Accumulator0, java.lang.String, java.lang.Integer) \n" + "Expected: (org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions" + ".Accumulator0, long, int)") -- Gitee From 00ee80313c9051a9c2dbe08d4d7528031b247a36 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Sat, 12 Oct 2019 14:09:37 +0800 Subject: [PATCH 186/268] [FLINK-14273][table-planner-blink] Add UserDefinedFunctionValidationTest to verify operand type check --- .../UserDefinedFunctionValidationTest.scala | 61 +++++++++++++++++++ 1 file changed, 61 insertions(+) create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala new file mode 100644 index 0000000000..953f0f3115 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala @@ -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.table.api.validation + +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.ValidationException +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.planner.expressions.utils.Func0 +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions.OverAgg0 +import org.apache.flink.table.planner.utils.TableTestBase +import org.junit.Test + +class UserDefinedFunctionValidationTest extends TableTestBase { + + @Test + def testScalarFunctionOperandTypeCheck(): Unit = { + thrown.expect(classOf[ValidationException]) + thrown.expectMessage( + "Given parameters of function 'func' do not match any signature. \n" + + "Actual: (java.lang.String) \n" + + "Expected: (int)") + val util = scalaStreamTestUtil() + util.addTableSource[(Int, String)]("t", 'a, 'b) + util.tableEnv.registerFunction("func", Func0) + util.verifyExplain("select func(b) from t") + } + + @Test + def testAggregateFunctionOperandTypeCheck(): Unit = { + thrown.expect(classOf[ValidationException]) + thrown.expectMessage( + "Given parameters of function 'agg' do not match any signature. \n" + + "Actual: (org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions" + + ".Accumulator0, java.lang.String, java.lang.Integer) \n" + + "Expected: (org.apache.flink.table.planner.runtime.utils.JavaUserDefinedAggFunctions" + + ".Accumulator0, long, int)") + + val util = scalaStreamTestUtil() + val agg = new OverAgg0 + util.addTableSource[(Int, String)]("t", 'a, 'b) + util.tableEnv.registerFunction("agg", agg) + util.verifyExplain("select agg(b, a) from t") + } + +} + -- Gitee From cee9bf0ab6c1d6b1678070ff2e05f0aa34f26dc0 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Thu, 10 Oct 2019 17:35:52 +0800 Subject: [PATCH 187/268] [FLINK-14004][runtime] Define SourceReaderOperator to verify the integration with StreamOneInputProcessor We already refactored the task input and output in runtime stack for considering the requirements of FLIP-27. In order to further verify that the new source could work well with the unified StreamOneInputProcessor in mailbox model, we define the SourceReaderOperator as task input and implement a unit test for passing through the whole process. --- .../api/operators/SourceReaderOperator.java | 34 ++++++ .../runtime/io/AbstractDataOutput.java | 51 +++++++++ .../runtime/io/StreamTaskSourceInput.java | 66 ++++++++++++ .../runtime/io/StreamTwoInputProcessor.java | 10 +- .../runtime/tasks/OneInputStreamTask.java | 19 +--- .../runtime/tasks/SourceReaderStreamTask.java | 100 ++++++++++++++++++ .../tasks/SourceReaderStreamTaskTest.java | 92 ++++++++++++++++ 7 files changed, 348 insertions(+), 24 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceReaderOperator.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractDataOutput.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskSourceInput.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceReaderStreamTask.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceReaderStreamTaskTest.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceReaderOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceReaderOperator.java new file mode 100644 index 0000000000..8b73d96652 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceReaderOperator.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.operators; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; + +/** + * Base source operator only used for integrating the source reader which is proposed by FLIP-27. It implements + * the interface of {@link PushingAsyncDataInput} for naturally compatible with one input processing in runtime + * stack. + * + *

Note: We are expecting this to be changed to the concrete class once SourceReader interface is introduced. + * + * @param The output type of the operator + */ +@Internal +public abstract class SourceReaderOperator extends AbstractStreamOperator implements PushingAsyncDataInput { +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractDataOutput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractDataOutput.java new file mode 100644 index 0000000000..7ef94e8cad --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/AbstractDataOutput.java @@ -0,0 +1,51 @@ +/* + * 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.runtime.io; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; +import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Base class for all data outputs. It implements the unified way of emitting stream status + * for both network and source outputs. + * + * @param The output type + */ +@Internal +public abstract class AbstractDataOutput implements PushingAsyncDataInput.DataOutput { + + /** The maintainer toggles the current stream status. */ + protected final StreamStatusMaintainer streamStatusMaintainer; + + protected final Object lock; + + public AbstractDataOutput(StreamStatusMaintainer streamStatusMaintainer, Object lock) { + this.streamStatusMaintainer = checkNotNull(streamStatusMaintainer); + this.lock = checkNotNull(lock); + } + + @Override + public void emitStreamStatus(StreamStatus streamStatus) { + synchronized (lock) { + streamStatusMaintainer.toggleStreamStatus(streamStatus); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskSourceInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskSourceInput.java new file mode 100644 index 0000000000..46878248e9 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskSourceInput.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.runtime.io; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.api.operators.SourceReaderOperator; +import org.apache.flink.util.IOUtils; + +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Implementation of {@link StreamTaskInput} that reads data from the {@link SourceReaderOperator} + * and returns the {@link InputStatus} to indicate whether the source state is available, + * unavailable or finished. + */ +@Internal +public final class StreamTaskSourceInput implements StreamTaskInput { + + private final SourceReaderOperator operator; + + public StreamTaskSourceInput(SourceReaderOperator operator) { + this.operator = checkNotNull(operator); + } + + @Override + public InputStatus emitNext(DataOutput output) throws Exception { + return operator.emitNext(output); + } + + @Override + public CompletableFuture isAvailable() { + return operator.isAvailable(); + } + + /** + * This method is invalid and never called by the one/source input processor. + */ + @Override + public int getInputIndex() { + return -1; + } + + @Override + public void close() { + IOUtils.closeQuietly(operator::close); + } +} + diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index 34e566f62a..a18defe8ed 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -332,18 +332,13 @@ public final class StreamTwoInputProcessor implements StreamInputProce * The network data output implementation used for processing stream elements * from {@link StreamTaskNetworkInput} in two input selective processor. */ - private class StreamTaskNetworkOutput implements DataOutput { + private class StreamTaskNetworkOutput extends AbstractDataOutput { private final TwoInputStreamOperator operator; /** The function way is only used for frequent record processing as for JIT optimization. */ private final ThrowingConsumer, Exception> recordConsumer; - private final Object lock; - - /** The maintainer toggles the current stream status as well as retrieves it. */ - private final StreamStatusMaintainer streamStatusMaintainer; - private final WatermarkGauge inputWatermarkGauge; /** The input index to indicate how to process elements by two input operator. */ @@ -356,11 +351,10 @@ public final class StreamTwoInputProcessor implements StreamInputProce StreamStatusMaintainer streamStatusMaintainer, WatermarkGauge inputWatermarkGauge, int inputIndex) { + super(streamStatusMaintainer, lock); this.operator = checkNotNull(operator); this.recordConsumer = checkNotNull(recordConsumer); - this.lock = checkNotNull(lock); - this.streamStatusMaintainer = checkNotNull(streamStatusMaintainer); this.inputWatermarkGauge = checkNotNull(inputWatermarkGauge); this.inputIndex = inputIndex; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java index 458cd06b2e..2610661d6c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java @@ -29,6 +29,7 @@ import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.AbstractDataOutput; import org.apache.flink.streaming.runtime.io.CheckpointedInputGate; import org.apache.flink.streaming.runtime.io.InputGateUtil; import org.apache.flink.streaming.runtime.io.InputProcessorUtil; @@ -40,7 +41,6 @@ import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve; -import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import javax.annotation.Nullable; @@ -145,15 +145,10 @@ public class OneInputStreamTask extends StreamTask implements DataOutput { + private static class StreamTaskNetworkOutput extends AbstractDataOutput { private final OneInputStreamOperator operator; - /** The maintainer toggles the current stream status. */ - private final StreamStatusMaintainer streamStatusMaintainer; - - private final Object lock; - private final WatermarkGauge watermarkGauge; private final Counter numRecordsIn; @@ -163,10 +158,9 @@ public class OneInputStreamTask extends StreamTask extends StreamTask extends StreamTask> { + + public SourceReaderStreamTask(Environment env) { + super(env); + } + + @Override + public void init() { + StreamTaskInput input = new StreamTaskSourceInput<>(headOperator); + DataOutput output = new StreamTaskSourceOutput<>( + operatorChain.getChainEntryPoint(), + getStreamStatusMaintainer(), + getCheckpointLock()); + + inputProcessor = new StreamOneInputProcessor<>( + input, + output, + getCheckpointLock(), + operatorChain); + } + + /** + * Implementation of {@link DataOutput} that wraps a specific {@link Output} to emit + * stream elements for {@link SourceReaderOperator}. + */ + private static class StreamTaskSourceOutput extends AbstractDataOutput { + + private final Output> output; + + StreamTaskSourceOutput( + Output> output, + StreamStatusMaintainer streamStatusMaintainer, + Object lock) { + super(streamStatusMaintainer, lock); + + this.output = checkNotNull(output); + } + + @Override + public void emitRecord(StreamRecord streamRecord) { + synchronized (lock) { + output.collect(streamRecord); + } + } + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) { + synchronized (lock) { + output.emitLatencyMarker(latencyMarker); + } + } + + @Override + public void emitWatermark(Watermark watermark) { + synchronized (lock) { + output.emitWatermark(watermark); + } + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceReaderStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceReaderStreamTaskTest.java new file mode 100644 index 0000000000..61934c6dcd --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceReaderStreamTaskTest.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.runtime.tasks; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.SourceReaderOperator; +import org.apache.flink.streaming.runtime.io.InputStatus; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.TestHarnessUtil; + +import org.junit.Test; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; + +/** + * Tests for verifying that the {@link SourceReaderOperator} as a task input can be integrated + * well with {@link org.apache.flink.streaming.runtime.io.StreamOneInputProcessor}. + */ +public class SourceReaderStreamTaskTest { + + @Test + public void testSourceOutputCorrectness() throws Exception { + final int numRecords = 10; + final StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>( + SourceReaderStreamTask::new, + BasicTypeInfo.INT_TYPE_INFO); + final StreamConfig streamConfig = testHarness.getStreamConfig(); + + testHarness.setupOutputForSingletonOperatorChain(); + streamConfig.setStreamOperator(new TestingFiniteSourceReaderOperator(numRecords)); + streamConfig.setOperatorID(new OperatorID()); + + testHarness.invoke(); + testHarness.waitForTaskCompletion(); + + final LinkedBlockingQueue expectedOutput = new LinkedBlockingQueue<>(); + for (int i = 1; i <= numRecords; i++) { + expectedOutput.add(new StreamRecord<>(i)); + } + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + } + + /** + * A simple {@link SourceReaderOperator} implementation for emitting limited int type records. + */ + private static class TestingFiniteSourceReaderOperator extends SourceReaderOperator { + private static final long serialVersionUID = 1L; + + private final int numRecords; + private int counter; + + TestingFiniteSourceReaderOperator(int numRecords) { + this.numRecords = numRecords; + } + + @Override + public InputStatus emitNext(DataOutput output) throws Exception { + output.emitRecord(new StreamRecord<>(++counter)); + + return counter < numRecords ? InputStatus.MORE_AVAILABLE : InputStatus.END_OF_INPUT; + } + + @Override + public CompletableFuture isAvailable() { + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + } + } +} -- Gitee From 345bfefd91c5a1a309ff1a4b397eb9882788fa2a Mon Sep 17 00:00:00 2001 From: Wei Zhong Date: Wed, 9 Oct 2019 16:33:56 +0800 Subject: [PATCH 188/268] [FLINK-14212][python] Support no-argument Python UDFs. Support Python UDFs with no arguments, e.g., tab.select("one(), two()"). This closes #9865. --- flink-python/pyflink/table/tests/test_udf.py | 16 +++ .../planner/codegen/ExpressionReducer.scala | 124 ++++++++++-------- .../functions/utils/ScalarSqlFunction.scala | 2 +- .../table/planner/plan/utils/PythonUtil.scala | 68 ++++++++++ .../logical/ExpressionReductionRulesTest.xml | 17 +++ .../ExpressionReductionRulesTest.scala | 16 +++ .../table/codegen/ExpressionReducer.scala | 69 ++++++---- .../flink/table/plan/util/PythonUtil.scala | 2 +- .../plan/ExpressionReductionRulesTest.scala | 29 +++- 9 files changed, 260 insertions(+), 83 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/PythonUtil.scala diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index bf46ea9a5d..d7864be749 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -204,6 +204,22 @@ class UserDefinedFunctionTests(PyFlinkStreamTableTestCase): self.t_env.register_function( "non-callable-udf", udf(Plus(), DataTypes.BIGINT(), DataTypes.BIGINT())) + def test_udf_without_arguments(self): + self.t_env.register_function("one", udf( + lambda: 1, input_types=[], result_type=DataTypes.BIGINT(), deterministic=True)) + self.t_env.register_function("two", udf( + lambda: 2, input_types=[], result_type=DataTypes.BIGINT(), deterministic=False)) + + table_sink = source_sink_utils.TestAppendSink(['a', 'b'], + [DataTypes.BIGINT(), DataTypes.BIGINT()]) + self.t_env.register_table_sink("Results", table_sink) + + t = self.t_env.from_elements([(1, 2), (2, 5), (3, 1)], ['a', 'b']) + t.select("one(), two()").insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, ["1,2", "1,2", "1,2"]) + @udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT()) def add(i, j): diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala index bb5ff90cbb..d12701193a 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala @@ -24,9 +24,10 @@ import org.apache.flink.metrics.MetricGroup import org.apache.flink.table.api.{TableConfig, TableException} import org.apache.flink.table.dataformat.BinaryStringUtil.safeToString import org.apache.flink.table.dataformat.{BinaryString, Decimal, GenericRow} -import org.apache.flink.table.functions.{FunctionContext, UserDefinedFunction} +import org.apache.flink.table.functions.{FunctionContext, FunctionLanguage, UserDefinedFunction} import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.codegen.FunctionCodeGenerator.generateFunction +import org.apache.flink.table.planner.plan.utils.PythonUtil import org.apache.flink.table.runtime.functions.SqlDateTimeUtils import org.apache.flink.table.types.logical.RowType @@ -39,6 +40,7 @@ import java.io.File import java.util.TimeZone import scala.collection.JavaConverters._ +import scala.collection.mutable.ListBuffer /** * Evaluates constant expressions with code generator. @@ -60,8 +62,16 @@ class ExpressionReducer( constExprs: java.util.List[RexNode], reducedValues: java.util.List[RexNode]): Unit = { + val pythonUDFExprs = new ListBuffer[RexNode]() + val literals = constExprs.asScala.map(e => (e.getType.getSqlTypeName, e)).flatMap { + // Skip expressions that contain python functions because it's quite expensive to + // call Python UDFs during optimization phase. They will be optimized during the runtime. + case (_, e) if PythonUtil.containsFunctionOf(e, FunctionLanguage.PYTHON) => + pythonUDFExprs += e + None + // we don't support object literals yet, we skip those constant expressions case (SqlTypeName.ANY, _) | (SqlTypeName.ROW, _) | @@ -120,59 +130,65 @@ class ExpressionReducer( var reducedIdx = 0 while (i < constExprs.size()) { val unreduced = constExprs.get(i) - unreduced.getType.getSqlTypeName match { - // we insert the original expression for object literals - case SqlTypeName.ANY | - SqlTypeName.ROW | - SqlTypeName.ARRAY | - SqlTypeName.MAP | - SqlTypeName.MULTISET => - reducedValues.add(unreduced) - case SqlTypeName.VARCHAR | SqlTypeName.CHAR => - val escapeVarchar = StringEscapeUtils - .escapeJava(safeToString(reduced.getField(reducedIdx).asInstanceOf[BinaryString])) - reducedValues.add(maySkipNullLiteralReduce(rexBuilder, escapeVarchar, unreduced)) - reducedIdx += 1 - case SqlTypeName.VARBINARY | SqlTypeName.BINARY => - val reducedValue = reduced.getField(reducedIdx) - val value = if (null != reducedValue) { - new ByteString(reduced.getField(reducedIdx).asInstanceOf[Array[Byte]]) - } else { - reducedValue - } - reducedValues.add(maySkipNullLiteralReduce(rexBuilder, value, unreduced)) - reducedIdx += 1 - case SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE => - val value = if (!reduced.isNullAt(reducedIdx)) { - val mills = reduced.getField(reducedIdx).asInstanceOf[Long] - Long.box(SqlDateTimeUtils.timestampWithLocalZoneToTimestamp( - mills, TimeZone.getTimeZone(config.getLocalTimeZone))) - } else { - null - } - reducedValues.add(maySkipNullLiteralReduce(rexBuilder, value, unreduced)) - reducedIdx += 1 - case SqlTypeName.DECIMAL => - val reducedValue = reduced.getField(reducedIdx) - val value = if (reducedValue != null) { - reducedValue.asInstanceOf[Decimal].toBigDecimal - } else { - reducedValue - } - reducedValues.add(maySkipNullLiteralReduce(rexBuilder, value, unreduced)) - reducedIdx += 1 - case _ => - val reducedValue = reduced.getField(reducedIdx) - // RexBuilder handle double literal incorrectly, convert it into BigDecimal manually - val value = if (reducedValue != null && - unreduced.getType.getSqlTypeName == SqlTypeName.DOUBLE) { - new java.math.BigDecimal(reducedValue.asInstanceOf[Number].doubleValue()) - } else { - reducedValue - } - - reducedValues.add(maySkipNullLiteralReduce(rexBuilder, value, unreduced)) - reducedIdx += 1 + // use eq to compare reference + if (pythonUDFExprs.exists(_ eq unreduced)) { + // if contains python function then just insert the original expression. + reducedValues.add(unreduced) + } else { + unreduced.getType.getSqlTypeName match { + // we insert the original expression for object literals + case SqlTypeName.ANY | + SqlTypeName.ROW | + SqlTypeName.ARRAY | + SqlTypeName.MAP | + SqlTypeName.MULTISET => + reducedValues.add(unreduced) + case SqlTypeName.VARCHAR | SqlTypeName.CHAR => + val escapeVarchar = StringEscapeUtils + .escapeJava(safeToString(reduced.getField(reducedIdx).asInstanceOf[BinaryString])) + reducedValues.add(maySkipNullLiteralReduce(rexBuilder, escapeVarchar, unreduced)) + reducedIdx += 1 + case SqlTypeName.VARBINARY | SqlTypeName.BINARY => + val reducedValue = reduced.getField(reducedIdx) + val value = if (null != reducedValue) { + new ByteString(reduced.getField(reducedIdx).asInstanceOf[Array[Byte]]) + } else { + reducedValue + } + reducedValues.add(maySkipNullLiteralReduce(rexBuilder, value, unreduced)) + reducedIdx += 1 + case SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE => + val value = if (!reduced.isNullAt(reducedIdx)) { + val mills = reduced.getField(reducedIdx).asInstanceOf[Long] + Long.box(SqlDateTimeUtils.timestampWithLocalZoneToTimestamp( + mills, TimeZone.getTimeZone(config.getLocalTimeZone))) + } else { + null + } + reducedValues.add(maySkipNullLiteralReduce(rexBuilder, value, unreduced)) + reducedIdx += 1 + case SqlTypeName.DECIMAL => + val reducedValue = reduced.getField(reducedIdx) + val value = if (reducedValue != null) { + reducedValue.asInstanceOf[Decimal].toBigDecimal + } else { + reducedValue + } + reducedValues.add(maySkipNullLiteralReduce(rexBuilder, value, unreduced)) + reducedIdx += 1 + case _ => + val reducedValue = reduced.getField(reducedIdx) + // RexBuilder handle double literal incorrectly, convert it into BigDecimal manually + val value = if (reducedValue != null && + unreduced.getType.getSqlTypeName == SqlTypeName.DOUBLE) { + new java.math.BigDecimal(reducedValue.asInstanceOf[Number].doubleValue()) + } else { + reducedValue + } + + reducedValues.add(maySkipNullLiteralReduce(rexBuilder, value, unreduced)) + reducedIdx += 1 + } } i += 1 } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/ScalarSqlFunction.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/ScalarSqlFunction.scala index 159d4f1dc0..8e5609caca 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/ScalarSqlFunction.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/ScalarSqlFunction.scala @@ -47,7 +47,7 @@ import scala.collection.JavaConverters._ class ScalarSqlFunction( name: String, displayName: String, - scalarFunction: ScalarFunction, + val scalarFunction: ScalarFunction, typeFactory: FlinkTypeFactory, returnTypeInfer: Option[SqlReturnTypeInference] = None) extends SqlFunction( diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/PythonUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/PythonUtil.scala new file mode 100644 index 0000000000..141d1dc3cb --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/PythonUtil.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.utils + +import org.apache.calcite.rex.{RexCall, RexNode} +import org.apache.flink.table.functions.FunctionLanguage +import org.apache.flink.table.planner.functions.utils.ScalarSqlFunction + +import scala.collection.JavaConversions._ + +object PythonUtil { + /** + * Checks whether it contains the specified kind of function in the specified node. + * + * @param node the RexNode to check + * @param language the expected kind of function to find + * @param recursive whether check the inputs of the specified node + * @return true if it contains the specified kind of function in the specified node. + */ + def containsFunctionOf( + node: RexNode, + language: FunctionLanguage, + recursive: Boolean = true): Boolean = { + node.accept(new FunctionFinder(language, recursive)) + } + + /** + * Checks whether it contains the specified kind of function in a RexNode. + * + * @param expectedLanguage the expected kind of function to find + * @param recursive whether check the inputs + */ + private class FunctionFinder(expectedLanguage: FunctionLanguage, recursive: Boolean) + extends RexDefaultVisitor[Boolean] { + + override def visitCall(call: RexCall): Boolean = { + call.getOperator match { + case sfc: ScalarSqlFunction if sfc.scalarFunction.getLanguage == + FunctionLanguage.PYTHON => + findInternal(FunctionLanguage.PYTHON, call) + case _ => + findInternal(FunctionLanguage.JVM, call) + } + } + + override def visitNode(rexNode: RexNode): Boolean = false + + private def findInternal(actualLanguage: FunctionLanguage, call: RexCall): Boolean = + actualLanguage == expectedLanguage || + (recursive && call.getOperands.exists(_.accept(this))) + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.xml index 9e301578e5..0433fbee48 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.xml @@ -47,6 +47,23 @@ LogicalProject(EXPR$0=[myUdf(1)]) + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala index 79cc13b7d3..54541ba3fa 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.rules.logical import org.apache.flink.api.scala._ import org.apache.flink.table.api.scala._ +import org.apache.flink.table.functions.{FunctionLanguage, ScalarFunction} import org.apache.flink.table.planner.expressions.utils.{Func1, RichFunc1} import org.apache.flink.table.planner.utils.TableTestBase @@ -46,4 +47,19 @@ class ExpressionReductionRulesTest extends TableTestBase { util.verifyPlan("SELECT myUdf(1) FROM MyTable") } + @Test + def testExpressionReductionWithPythonUDF(): Unit = { + util.addFunction("PyUdf", DeterministicPythonFunc) + util.addFunction("MyUdf", Func1) + util.verifyPlan("SELECT PyUdf(), MyUdf(1) FROM MyTable") + } +} + +object DeterministicPythonFunc extends ScalarFunction { + + override def getLanguage: FunctionLanguage = FunctionLanguage.PYTHON + + def eval(): Long = 1 + + override def isDeterministic = true } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala index dfed70be10..e3e44ae8a1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala @@ -28,9 +28,12 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.api.TableConfig import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.functions.FunctionLanguage +import org.apache.flink.table.plan.util.PythonUtil import org.apache.flink.types.Row import scala.collection.JavaConverters._ +import scala.collection.mutable.ListBuffer /** * Evaluates constant expressions using Flink's [[FunctionCodeGenerator]]. @@ -48,8 +51,16 @@ class ExpressionReducer(config: TableConfig) val typeFactory = rexBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory] + val pythonUDFExprs = ListBuffer[RexNode]() + val literals = constExprs.asScala.map(e => (e.getType.getSqlTypeName, e)).flatMap { + // Skip expressions that contain python functions because it's quite expensive to + // call Python UDFs during optimization phase. They will be optimized during the runtime. + case (_, e) if PythonUtil.containsFunctionOf(e, FunctionLanguage.PYTHON) => + pythonUDFExprs += e + None + // we need to cast here for RexBuilder.makeLiteral case (SqlTypeName.DATE, e) => Some( @@ -114,34 +125,40 @@ class ExpressionReducer(config: TableConfig) var reducedIdx = 0 while (i < constExprs.size()) { val unreduced = constExprs.get(i) - unreduced.getType.getSqlTypeName match { - // we insert the original expression for object literals - case SqlTypeName.ANY | - SqlTypeName.ROW | - SqlTypeName.ARRAY | - SqlTypeName.MAP | - SqlTypeName.MULTISET => - reducedValues.add(unreduced) - - case _ => - val reducedValue = reduced.getField(reducedIdx) - // RexBuilder handle double literal incorrectly, convert it into BigDecimal manually - val value = if (unreduced.getType.getSqlTypeName == SqlTypeName.DOUBLE) { - if (reducedValue == null) { - reducedValue + // use eq to compare reference + if (pythonUDFExprs.exists(_ eq unreduced)) { + // if contains python function then just insert the original expression. + reducedValues.add(unreduced) + } else { + unreduced.getType.getSqlTypeName match { + // we insert the original expression for object literals + case SqlTypeName.ANY | + SqlTypeName.ROW | + SqlTypeName.ARRAY | + SqlTypeName.MAP | + SqlTypeName.MULTISET => + reducedValues.add(unreduced) + + case _ => + val reducedValue = reduced.getField(reducedIdx) + // RexBuilder handle double literal incorrectly, convert it into BigDecimal manually + val value = if (unreduced.getType.getSqlTypeName == SqlTypeName.DOUBLE) { + if (reducedValue == null) { + reducedValue + } else { + new java.math.BigDecimal(reducedValue.asInstanceOf[Number].doubleValue()) + } } else { - new java.math.BigDecimal(reducedValue.asInstanceOf[Number].doubleValue()) + reducedValue } - } else { - reducedValue - } - - val literal = rexBuilder.makeLiteral( - value, - unreduced.getType, - true) - reducedValues.add(literal) - reducedIdx += 1 + + val literal = rexBuilder.makeLiteral( + value, + unreduced.getType, + true) + reducedValues.add(literal) + reducedIdx += 1 + } } i += 1 } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/PythonUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/PythonUtil.scala index cd4efd9c77..2e0073cc12 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/PythonUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/PythonUtil.scala @@ -46,7 +46,7 @@ object PythonUtil { * @param expectedLanguage the expected kind of function to find * @param recursive whether check the inputs */ - class FunctionFinder(expectedLanguage: FunctionLanguage, recursive: Boolean) + private class FunctionFinder(expectedLanguage: FunctionLanguage, recursive: Boolean) extends RexDefaultVisitor[Boolean] { override def visitCall(call: RexCall): Boolean = { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala index f3cc8caf4f..ce29bed319 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala @@ -21,7 +21,7 @@ package org.apache.flink.table.plan 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.functions.ScalarFunction +import org.apache.flink.table.functions.{FunctionLanguage, ScalarFunction} import org.apache.flink.table.utils.TableTestBase import org.apache.flink.table.utils.TableTestUtil._ import org.junit.{Ignore, Test} @@ -499,6 +499,24 @@ class ExpressionReductionRulesTest extends TableTestBase { util.verifyTable(result, expected) } + + @Test + def testReduceDeterministicPythonUDF(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c) + + val result = table + .select('a, 'b, 'c, DeterministicPythonFunc() as 'd, DeterministicNullFunc() as 'e) + + val expected: String = unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "a", "b", "c", "DeterministicPythonFunc$() AS d", + "null:VARCHAR(65536) AS e") + ) + + util.verifyTable(result, expected) + } } object NonDeterministicNullFunc extends ScalarFunction { @@ -510,3 +528,12 @@ object DeterministicNullFunc extends ScalarFunction { def eval(): String = null override def isDeterministic = true } + +object DeterministicPythonFunc extends ScalarFunction { + + override def getLanguage: FunctionLanguage = FunctionLanguage.PYTHON + + def eval(): Long = 1L + + override def isDeterministic = true +} -- Gitee From ddfed72813281255f119fd6838c197433eb6eaf3 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Wed, 9 Oct 2019 11:38:35 +0800 Subject: [PATCH 189/268] [FLINK-14208][python] Support Python UDFs with parameters of constant values Support Python UDFs with parameters of constant values. The constant parameters are not needed to be transferred between the Java operator and the Python worker for each input record. This closes #9858. --- .../fn_execution/flink_fn_execution_pb2.py | 42 ++++--- .../pyflink/fn_execution/operations.py | 45 +++++++- .../pyflink/proto/flink-fn-execution.proto | 2 + flink-python/pyflink/table/tests/test_udf.py | 106 ++++++++++++++++++ .../api/common/python/PythonBridgeUtils.java | 75 +++++++++++++ .../AbstractPythonScalarFunctionRunner.java | 4 +- .../table/plan/nodes/CommonPythonCalc.scala | 20 +++- 7 files changed, 270 insertions(+), 24 deletions(-) diff --git a/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py index b473673460..f6106a0fb1 100644 --- a/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py +++ b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py @@ -36,7 +36,7 @@ DESCRIPTOR = _descriptor.FileDescriptor( name='flink-fn-execution.proto', package='org.apache.flink.fn_execution.v1', syntax='proto3', - serialized_pb=_b('\n\x18\x66link-fn-execution.proto\x12 org.apache.flink.fn_execution.v1\"\xe2\x01\n\x13UserDefinedFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12K\n\x06inputs\x18\x02 \x03(\x0b\x32;.org.apache.flink.fn_execution.v1.UserDefinedFunction.Input\x1am\n\x05Input\x12\x44\n\x03udf\x18\x01 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunctionH\x00\x12\x15\n\x0binputOffset\x18\x02 \x01(\x05H\x00\x42\x07\n\x05input\"[\n\x14UserDefinedFunctions\x12\x43\n\x04udfs\x18\x01 \x03(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunction\"\x8d\x07\n\x06Schema\x12>\n\x06\x66ields\x18\x01 \x03(\x0b\x32..org.apache.flink.fn_execution.v1.Schema.Field\x1a\x97\x01\n\x07MapType\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\xcd\x02\n\tFieldType\x12\x44\n\ttype_name\x18\x01 \x01(\x0e\x32\x31.org.apache.flink.fn_execution.v1.Schema.TypeName\x12\x10\n\x08nullable\x18\x02 \x01(\x08\x12U\n\x17\x63ollection_element_type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeH\x00\x12\x44\n\x08map_type\x18\x04 \x01(\x0b\x32\x30.org.apache.flink.fn_execution.v1.Schema.MapTypeH\x00\x12>\n\nrow_schema\x18\x05 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x42\x0b\n\ttype_info\x1al\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12@\n\x04type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\"\xea\x01\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\x0b\n\x07TINYINT\x10\x01\x12\x0c\n\x08SMALLINT\x10\x02\x12\x07\n\x03INT\x10\x03\x12\n\n\x06\x42IGINT\x10\x04\x12\x0b\n\x07\x44\x45\x43IMAL\x10\x05\x12\t\n\x05\x46LOAT\x10\x06\x12\n\n\x06\x44OUBLE\x10\x07\x12\x08\n\x04\x44\x41TE\x10\x08\x12\x08\n\x04TIME\x10\t\x12\x0c\n\x08\x44\x41TETIME\x10\n\x12\x0b\n\x07\x42OOLEAN\x10\x0b\x12\n\n\x06\x42INARY\x10\x0c\x12\r\n\tVARBINARY\x10\r\x12\x08\n\x04\x43HAR\x10\x0e\x12\x0b\n\x07VARCHAR\x10\x0f\x12\t\n\x05\x41RRAY\x10\x10\x12\x07\n\x03MAP\x10\x11\x12\x0c\n\x08MULTISET\x10\x12\x42-\n\x1forg.apache.flink.fnexecution.v1B\nFlinkFnApib\x06proto3') + serialized_pb=_b('\n\x18\x66link-fn-execution.proto\x12 org.apache.flink.fn_execution.v1\"\xfc\x01\n\x13UserDefinedFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12K\n\x06inputs\x18\x02 \x03(\x0b\x32;.org.apache.flink.fn_execution.v1.UserDefinedFunction.Input\x1a\x86\x01\n\x05Input\x12\x44\n\x03udf\x18\x01 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunctionH\x00\x12\x15\n\x0binputOffset\x18\x02 \x01(\x05H\x00\x12\x17\n\rinputConstant\x18\x03 \x01(\x0cH\x00\x42\x07\n\x05input\"[\n\x14UserDefinedFunctions\x12\x43\n\x04udfs\x18\x01 \x03(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunction\"\x8d\x07\n\x06Schema\x12>\n\x06\x66ields\x18\x01 \x03(\x0b\x32..org.apache.flink.fn_execution.v1.Schema.Field\x1a\x97\x01\n\x07MapType\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\xcd\x02\n\tFieldType\x12\x44\n\ttype_name\x18\x01 \x01(\x0e\x32\x31.org.apache.flink.fn_execution.v1.Schema.TypeName\x12\x10\n\x08nullable\x18\x02 \x01(\x08\x12U\n\x17\x63ollection_element_type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeH\x00\x12\x44\n\x08map_type\x18\x04 \x01(\x0b\x32\x30.org.apache.flink.fn_execution.v1.Schema.MapTypeH\x00\x12>\n\nrow_schema\x18\x05 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x42\x0b\n\ttype_info\x1al\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12@\n\x04type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\"\xea\x01\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\x0b\n\x07TINYINT\x10\x01\x12\x0c\n\x08SMALLINT\x10\x02\x12\x07\n\x03INT\x10\x03\x12\n\n\x06\x42IGINT\x10\x04\x12\x0b\n\x07\x44\x45\x43IMAL\x10\x05\x12\t\n\x05\x46LOAT\x10\x06\x12\n\n\x06\x44OUBLE\x10\x07\x12\x08\n\x04\x44\x41TE\x10\x08\x12\x08\n\x04TIME\x10\t\x12\x0c\n\x08\x44\x41TETIME\x10\n\x12\x0b\n\x07\x42OOLEAN\x10\x0b\x12\n\n\x06\x42INARY\x10\x0c\x12\r\n\tVARBINARY\x10\r\x12\x08\n\x04\x43HAR\x10\x0e\x12\x0b\n\x07VARCHAR\x10\x0f\x12\t\n\x05\x41RRAY\x10\x10\x12\x07\n\x03MAP\x10\x11\x12\x0c\n\x08MULTISET\x10\x12\x42-\n\x1forg.apache.flink.fnexecution.v1B\nFlinkFnApib\x06proto3') ) @@ -126,8 +126,8 @@ _SCHEMA_TYPENAME = _descriptor.EnumDescriptor( ], containing_type=None, options=None, - serialized_start=1060, - serialized_end=1294, + serialized_start=1086, + serialized_end=1320, ) _sym_db.RegisterEnumDescriptor(_SCHEMA_TYPENAME) @@ -153,6 +153,13 @@ _USERDEFINEDFUNCTION_INPUT = _descriptor.Descriptor( message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='inputConstant', full_name='org.apache.flink.fn_execution.v1.UserDefinedFunction.Input.inputConstant', index=2, + number=3, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), ], extensions=[ ], @@ -168,8 +175,8 @@ _USERDEFINEDFUNCTION_INPUT = _descriptor.Descriptor( name='input', full_name='org.apache.flink.fn_execution.v1.UserDefinedFunction.Input.input', index=0, containing_type=None, fields=[]), ], - serialized_start=180, - serialized_end=289, + serialized_start=181, + serialized_end=315, ) _USERDEFINEDFUNCTION = _descriptor.Descriptor( @@ -206,7 +213,7 @@ _USERDEFINEDFUNCTION = _descriptor.Descriptor( oneofs=[ ], serialized_start=63, - serialized_end=289, + serialized_end=315, ) @@ -236,8 +243,8 @@ _USERDEFINEDFUNCTIONS = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=291, - serialized_end=382, + serialized_start=317, + serialized_end=408, ) @@ -274,8 +281,8 @@ _SCHEMA_MAPTYPE = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=460, - serialized_end=611, + serialized_start=486, + serialized_end=637, ) _SCHEMA_FIELDTYPE = _descriptor.Descriptor( @@ -335,8 +342,8 @@ _SCHEMA_FIELDTYPE = _descriptor.Descriptor( name='type_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.type_info', index=0, containing_type=None, fields=[]), ], - serialized_start=614, - serialized_end=947, + serialized_start=640, + serialized_end=973, ) _SCHEMA_FIELD = _descriptor.Descriptor( @@ -379,8 +386,8 @@ _SCHEMA_FIELD = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=949, - serialized_end=1057, + serialized_start=975, + serialized_end=1083, ) _SCHEMA = _descriptor.Descriptor( @@ -410,8 +417,8 @@ _SCHEMA = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=385, - serialized_end=1294, + serialized_start=411, + serialized_end=1320, ) _USERDEFINEDFUNCTION_INPUT.fields_by_name['udf'].message_type = _USERDEFINEDFUNCTION @@ -422,6 +429,9 @@ _USERDEFINEDFUNCTION_INPUT.fields_by_name['udf'].containing_oneof = _USERDEFINED _USERDEFINEDFUNCTION_INPUT.oneofs_by_name['input'].fields.append( _USERDEFINEDFUNCTION_INPUT.fields_by_name['inputOffset']) _USERDEFINEDFUNCTION_INPUT.fields_by_name['inputOffset'].containing_oneof = _USERDEFINEDFUNCTION_INPUT.oneofs_by_name['input'] +_USERDEFINEDFUNCTION_INPUT.oneofs_by_name['input'].fields.append( + _USERDEFINEDFUNCTION_INPUT.fields_by_name['inputConstant']) +_USERDEFINEDFUNCTION_INPUT.fields_by_name['inputConstant'].containing_oneof = _USERDEFINEDFUNCTION_INPUT.oneofs_by_name['input'] _USERDEFINEDFUNCTION.fields_by_name['inputs'].message_type = _USERDEFINEDFUNCTION_INPUT _USERDEFINEDFUNCTIONS.fields_by_name['udfs'].message_type = _USERDEFINEDFUNCTION _SCHEMA_MAPTYPE.fields_by_name['key_type'].message_type = _SCHEMA_FIELDTYPE diff --git a/flink-python/pyflink/fn_execution/operations.py b/flink-python/pyflink/fn_execution/operations.py index 13e179bb53..4bd955cc71 100644 --- a/flink-python/pyflink/fn_execution/operations.py +++ b/flink-python/pyflink/fn_execution/operations.py @@ -16,6 +16,7 @@ # limitations under the License. ################################################################################ +import datetime from abc import abstractmethod, ABCMeta from apache_beam.runners.worker import operation_specs @@ -23,6 +24,7 @@ from apache_beam.runners.worker import bundle_processor from apache_beam.runners.worker.operations import Operation from pyflink.fn_execution import flink_fn_execution_pb2 +from pyflink.serializers import PickleSerializer SCALAR_FUNCTION_URN = "flink:transform:scalar_function:v1" @@ -79,6 +81,44 @@ class ScalarFunctionInputGetter(InputGetter): return self.scalar_function_invoker.invoke_eval(value) +class ConstantInputGetter(InputGetter): + """ + InputGetter for the input argument which is a constant value. + + :param constant_value: the constant value of the column + """ + + def __init__(self, constant_value): + j_type = constant_value[0] + serializer = PickleSerializer() + pickled_data = serializer.loads(constant_value[1:]) + # the type set contains + # TINYINT,SMALLINT,INTEGER,BIGINT,FLOAT,DOUBLE,DECIMAL,CHAR,VARCHAR,NULL,BOOLEAN + # the pickled_data doesn't need to transfer to anther python object + if j_type == '\x00' or j_type == 0: + self._constant_value = pickled_data + # the type is DATE + elif j_type == '\x01' or j_type == 1: + self._constant_value = \ + datetime.date(year=1970, month=1, day=1) + datetime.timedelta(days=pickled_data) + # the type is TIME + elif j_type == '\x02' or j_type == 2: + seconds, milliseconds = divmod(pickled_data, 1000) + minutes, seconds = divmod(seconds, 60) + hours, minutes = divmod(minutes, 60) + self._constant_value = datetime.time(hours, minutes, seconds, milliseconds * 1000) + # the type is TIMESTAMP + elif j_type == '\x03' or j_type == 3: + self._constant_value = \ + datetime.datetime(year=1970, month=1, day=1, hour=0, minute=0, second=0) \ + + datetime.timedelta(milliseconds=pickled_data) + else: + raise Exception("Unknown type %s, should never happen" % str(j_type)) + + def get(self, value): + return self._constant_value + + class ScalarFunctionInvoker(object): """ An abstraction that can be used to execute :class:`ScalarFunction` methods. @@ -97,9 +137,12 @@ class ScalarFunctionInvoker(object): if input.HasField("udf"): # for chaining Python UDF input: the input argument is a Python ScalarFunction self.input_getters.append(ScalarFunctionInputGetter(input.udf)) - else: + elif input.HasField("inputOffset"): # the input argument is a column of the input row self.input_getters.append(OffsetInputGetter(input.inputOffset)) + else: + # the input argument is a constant value + self.input_getters.append(ConstantInputGetter(input.inputConstant)) def invoke_open(self): """ diff --git a/flink-python/pyflink/proto/flink-fn-execution.proto b/flink-python/pyflink/proto/flink-fn-execution.proto index db6582d39d..4ce9ed3ceb 100644 --- a/flink-python/pyflink/proto/flink-fn-execution.proto +++ b/flink-python/pyflink/proto/flink-fn-execution.proto @@ -32,6 +32,7 @@ message UserDefinedFunction { oneof input { UserDefinedFunction udf = 1; int32 inputOffset = 2; + bytes inputConstant = 3; } } @@ -41,6 +42,7 @@ message UserDefinedFunction { // The input arguments of the user-defined function, it could be one of the following: // 1. A column from the input row // 2. The result of another user-defined function + // 3. The constant value of the column repeated Input inputs = 2; } diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index d7864be749..321cd784f2 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -112,6 +112,112 @@ class UserDefinedFunctionTests(PyFlinkStreamTableTestCase): actual = source_sink_utils.results() self.assert_equals(actual, ["2,Hi,2,Flink"]) + def test_udf_with_constant_params(self): + def udf_with_constant_params(p, null_param, tinyint_param, smallint_param, int_param, + bigint_param, decimal_param, float_param, double_param, + boolean_param, str_param, + date_param, time_param, timestamp_param): + # decide whether two floats are equal + def float_equal(a, b, rel_tol=1e-09, abs_tol=0.0): + return abs(a - b) <= max(rel_tol * max(abs(a), abs(b)), abs_tol) + + from decimal import Decimal + import datetime + + assert null_param is None, 'null_param is wrong value %s' % null_param + + assert isinstance(tinyint_param, int), 'tinyint_param of wrong type %s !' \ + % type(tinyint_param) + p += tinyint_param + assert isinstance(smallint_param, int), 'smallint_param of wrong type %s !' \ + % type(smallint_param) + p += smallint_param + assert isinstance(int_param, int), 'int_param of wrong type %s !' \ + % type(int_param) + p += int_param + assert isinstance(bigint_param, int), 'bigint_param of wrong type %s !' \ + % type(bigint_param) + p += bigint_param + assert decimal_param == Decimal('1.05'), \ + 'decimal_param is wrong value %s ' % decimal_param + + p += int(decimal_param) + + assert isinstance(float_param, float) and float_equal(float_param, 1.23, 1e-06), \ + 'float_param is wrong value %s ' % float_param + + p += int(float_param) + assert isinstance(double_param, float) and float_equal(double_param, 1.98932, 1e-07), \ + 'double_param is wrong value %s ' % double_param + + p += int(double_param) + + assert boolean_param is True, 'boolean_param is wrong value %s' % boolean_param + + assert str_param == 'flink', 'str_param is wrong value %s' % str_param + + assert date_param == datetime.date(year=2014, month=9, day=13), \ + 'date_param is wrong value %s' % date_param + + assert time_param == datetime.time(hour=12, minute=0, second=0), \ + 'time_param is wrong value %s' % time_param + + assert timestamp_param == datetime.datetime(1999, 9, 10, 5, 20, 10), \ + 'timestamp_param is wrong value %s' % timestamp_param + + return p + + self.t_env.register_function("udf_with_constant_params", + udf(udf_with_constant_params, + input_types=[DataTypes.BIGINT(), + DataTypes.BIGINT(), + DataTypes.TINYINT(), + DataTypes.SMALLINT(), + DataTypes.INT(), + DataTypes.BIGINT(), + DataTypes.DECIMAL(20, 10), + DataTypes.FLOAT(), + DataTypes.DOUBLE(), + DataTypes.BOOLEAN(), + DataTypes.STRING(), + DataTypes.DATE(), + DataTypes.TIME(), + DataTypes.TIMESTAMP()], + result_type=DataTypes.BIGINT())) + + self.t_env.register_function( + "udf_with_all_constant_params", udf(lambda i, j: i + j, + [DataTypes.BIGINT(), DataTypes.BIGINT()], + DataTypes.BIGINT())) + + table_sink = source_sink_utils.TestAppendSink(['a', 'b'], + [DataTypes.BIGINT(), DataTypes.BIGINT()]) + self.t_env.register_table_sink("Results", table_sink) + + t = self.t_env.from_elements([(1, 2, 3), (2, 5, 6), (3, 1, 9)], ['a', 'b', 'c']) + self.t_env.register_table("test_table", t) + self.t_env.sql_query("select udf_with_all_constant_params(" + "cast (1 as BIGINT)," + "cast (2 as BIGINT)), " + "udf_with_constant_params(a, " + "cast (null as BIGINT)," + "cast (1 as TINYINT)," + "cast (1 as SMALLINT)," + "cast (1 as INT)," + "cast (1 as BIGINT)," + "cast (1.05 as DECIMAL)," + "cast (1.23 as FLOAT)," + "cast (1.98932 as DOUBLE)," + "true," + "'flink'," + "cast ('2014-09-13' as DATE)," + "cast ('12:00:00' as TIME)," + "cast ('1999-9-10 05:20:10' as TIMESTAMP))" + " from test_table").insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, ["3,8", "3,9", "3,10"]) + def test_overwrite_builtin_function(self): self.t_env.register_function( "plus", udf(lambda i, j: i + j - 1, diff --git a/flink-python/src/main/java/org/apache/flink/api/common/python/PythonBridgeUtils.java b/flink-python/src/main/java/org/apache/flink/api/common/python/PythonBridgeUtils.java index 44a568b2f5..31c18f2243 100644 --- a/flink-python/src/main/java/org/apache/flink/api/common/python/PythonBridgeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/api/common/python/PythonBridgeUtils.java @@ -20,12 +20,16 @@ package org.apache.flink.api.common.python; import org.apache.flink.api.common.python.pickle.ArrayConstructor; import org.apache.flink.api.common.python.pickle.ByteArrayConstructor; +import net.razorvine.pickle.Pickler; import net.razorvine.pickle.Unpickler; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.sql.type.SqlTypeName; import java.io.DataInputStream; import java.io.EOFException; import java.io.FileInputStream; import java.io.IOException; +import java.math.BigDecimal; import java.util.ArrayList; import java.util.LinkedList; import java.util.List; @@ -70,6 +74,77 @@ public final class PythonBridgeUtils { return unpickledData; } + public static byte[] convertLiteralToPython(RexLiteral o, SqlTypeName typeName) { + byte type; + Object value; + Pickler pickler = new Pickler(); + if (o.getValue3() == null) { + type = 0; + value = null; + } else { + switch (typeName) { + case TINYINT: + type = 0; + value = ((BigDecimal) o.getValue3()).byteValueExact(); + break; + case SMALLINT: + type = 0; + value = ((BigDecimal) o.getValue3()).shortValueExact(); + break; + case INTEGER: + type = 0; + value = ((BigDecimal) o.getValue3()).intValueExact(); + break; + case BIGINT: + type = 0; + value = ((BigDecimal) o.getValue3()).longValueExact(); + break; + case FLOAT: + type = 0; + value = ((BigDecimal) o.getValue3()).floatValue(); + break; + case DOUBLE: + type = 0; + value = ((BigDecimal) o.getValue3()).doubleValue(); + break; + case DECIMAL: + case BOOLEAN: + type = 0; + value = o.getValue3(); + break; + case CHAR: + case VARCHAR: + type = 0; + value = o.getValue3().toString(); + break; + case DATE: + type = 1; + value = o.getValue3(); + break; + case TIME: + type = 2; + value = o.getValue3(); + break; + case TIMESTAMP: + type = 3; + value = o.getValue3(); + break; + default: + throw new RuntimeException("Unsupported type " + typeName); + } + } + byte[] pickledData; + try { + pickledData = pickler.dumps(value); + } catch (IOException e) { + throw new RuntimeException("Pickle Java object failed", e); + } + byte[] typePickledData = new byte[pickledData.length + 1]; + typePickledData[0] = type; + System.arraycopy(pickledData, 0, typePickledData, 1, pickledData.length); + return typePickledData; + } + private static List readPickledBytes(final String fileName) throws IOException { List objs = new LinkedList<>(); try (DataInputStream din = new DataInputStream(new FileInputStream(fileName))) { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonScalarFunctionRunner.java index 2679156378..69c178b0ad 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonScalarFunctionRunner.java @@ -183,8 +183,10 @@ public abstract class AbstractPythonScalarFunctionRunner extends Abstra FlinkFnApi.UserDefinedFunction.Input.newBuilder(); if (input instanceof PythonFunctionInfo) { inputProto.setUdf(getUserDefinedFunctionProto((PythonFunctionInfo) input)); - } else { + } else if (input instanceof Integer) { inputProto.setInputOffset((Integer) input); + } else { + inputProto.setInputConstant(ByteString.copyFrom((byte[]) input)); } builder.addInputs(inputProto); } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCalc.scala index 04f4c20a75..aca06762ed 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCalc.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.plan.nodes import org.apache.calcite.rex.{RexCall, RexInputRef, RexLiteral, RexNode} +import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.flink.table.functions.FunctionLanguage import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo, SimplePythonFunction} import org.apache.flink.table.functions.utils.ScalarSqlFunction @@ -27,18 +28,21 @@ import scala.collection.mutable trait CommonPythonCalc { + private lazy val convertLiteralToPython = { + val clazz = Class.forName("org.apache.flink.api.common.python.PythonBridgeUtils") + clazz.getMethod("convertLiteralToPython", classOf[RexLiteral], classOf[SqlTypeName]) + } + private[flink] def extractPythonScalarFunctionInfos( rexCalls: Array[RexCall]): (Array[Int], Array[PythonFunctionInfo]) = { // using LinkedHashMap to keep the insert order val inputNodes = new mutable.LinkedHashMap[RexNode, Integer]() val pythonFunctionInfos = rexCalls.map(createPythonScalarFunctionInfo(_, inputNodes)) - val udfInputOffsets = inputNodes.toArray.map(_._1).map { - case inputRef: RexInputRef => inputRef.getIndex - case _: RexLiteral => throw new Exception( - "Constants cannot be used as parameters of Python UDF for now. " + - "It will be supported in FLINK-14208") - } + val udfInputOffsets = inputNodes.toArray + .map(_._1) + .filter(_.isInstanceOf[RexInputRef]) + .map(_.asInstanceOf[RexInputRef].getIndex) (udfInputOffsets, pythonFunctionInfos) } @@ -54,6 +58,10 @@ trait CommonPythonCalc { val argPythonInfo = createPythonScalarFunctionInfo(pythonRexCall, inputNodes) inputs.append(argPythonInfo) + case literal: RexLiteral => + inputs.append( + convertLiteralToPython.invoke(null, literal, literal.getType.getSqlTypeName)) + case argNode: RexNode => // For input arguments of RexInputRef, it's replaced with an offset into the input row inputNodes.get(argNode) match { -- Gitee From aaebe3b75f0ea7a28c5449168d514c381f12c6b5 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Mon, 7 Oct 2019 10:45:53 +0200 Subject: [PATCH 190/268] [hotfix][runtime] Make SchedulerBase#getInputsLocationsRetriever() final --- .../java/org/apache/flink/runtime/scheduler/SchedulerBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java index 4051ed4f32..0bddad0fcd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java @@ -305,7 +305,7 @@ public abstract class SchedulerBase implements SchedulerNG { return schedulingTopology; } - protected InputsLocationsRetriever getInputsLocationsRetriever() { + protected final InputsLocationsRetriever getInputsLocationsRetriever() { return inputsLocationsRetriever; } -- Gitee From 390428dca4cf218f57c453adb70aafcbd63b3e16 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Mon, 7 Oct 2019 10:47:00 +0200 Subject: [PATCH 191/268] [hotfix][runtime] Make SchedulerBase#getFailoverTopology() final --- .../java/org/apache/flink/runtime/scheduler/SchedulerBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java index 0bddad0fcd..b4fd647015 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java @@ -297,7 +297,7 @@ public abstract class SchedulerBase implements SchedulerNG { executionGraph.failJob(cause); } - protected FailoverTopology getFailoverTopology() { + protected final FailoverTopology getFailoverTopology() { return failoverTopology; } -- Gitee From e39a556be6c766b77305859f8cf6fda073af8017 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Mon, 7 Oct 2019 10:47:34 +0200 Subject: [PATCH 192/268] [hotfix][runtime] Make SchedulerBase#getSchedulingTopology() final --- .../java/org/apache/flink/runtime/scheduler/SchedulerBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java index b4fd647015..1b4f62edcf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java @@ -301,7 +301,7 @@ public abstract class SchedulerBase implements SchedulerNG { return failoverTopology; } - protected SchedulingTopology getSchedulingTopology() { + protected final SchedulingTopology getSchedulingTopology() { return schedulingTopology; } -- Gitee From afd05d3fd7e10445c3d9363bf541538d003b0c35 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Wed, 2 Oct 2019 17:53:18 +0200 Subject: [PATCH 193/268] [hotfix][tests] Remove unused methods from FailingExecutionVertexOperationsDecorator --- .../FailingExecutionVertexOperationsDecorator.java | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/FailingExecutionVertexOperationsDecorator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/FailingExecutionVertexOperationsDecorator.java index b5ad29f96b..76db6d6636 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/FailingExecutionVertexOperationsDecorator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/FailingExecutionVertexOperationsDecorator.java @@ -20,7 +20,6 @@ package org.apache.flink.runtime.scheduler; import org.apache.flink.runtime.JobException; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import java.util.concurrent.CompletableFuture; @@ -36,8 +35,6 @@ public class FailingExecutionVertexOperationsDecorator implements ExecutionVerte private boolean failDeploy; - private boolean failCancel; - public FailingExecutionVertexOperationsDecorator(final ExecutionVertexOperations delegate) { this.delegate = checkNotNull(delegate); } @@ -53,11 +50,7 @@ public class FailingExecutionVertexOperationsDecorator implements ExecutionVerte @Override public CompletableFuture cancel(final ExecutionVertex executionVertex) { - if (failCancel) { - return FutureUtils.completedExceptionally(new RuntimeException("Expected")); - } else { return delegate.cancel(executionVertex); - } } public void enableFailDeploy() { @@ -68,11 +61,4 @@ public class FailingExecutionVertexOperationsDecorator implements ExecutionVerte failDeploy = false; } - public void enableFailCancel() { - failCancel = true; - } - - public void disableFailCancel() { - failCancel = false; - } } -- Gitee From dbe1bfa31db4a561b6faa9c1235f02dc130825ca Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Fri, 4 Oct 2019 12:21:13 +0200 Subject: [PATCH 194/268] [FLINK-14291][runtime, tests] Add test coverage to DefaultScheduler - Remove SubmissionTrackingTaskManagerGateway, and introduce TestExecutionVertexOperationsDecorator to track task deployments. - Introduce DefaultExecutionSlotAllocatorFactory - Introduce TestExecutionSlotAllocator This closes #9872. --- .../DefaultExecutionSlotAllocatorFactory.java | 48 +++++++ .../runtime/scheduler/DefaultScheduler.java | 5 +- .../scheduler/DefaultSchedulerFactory.java | 3 +- .../ExecutionSlotAllocatorFactory.java | 29 ++++ .../scheduler/DefaultSchedulerTest.java | 101 +++++++++----- .../SubmissionTrackingTaskManagerGateway.java | 94 ------------- .../scheduler/TestExecutionSlotAllocator.java | 129 ++++++++++++++++++ .../TestExecutionSlotAllocatorFactory.java | 37 +++++ ...stExecutionVertexOperationsDecorator.java} | 24 +++- 9 files changed, 332 insertions(+), 138 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionSlotAllocatorFactory.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SubmissionTrackingTaskManagerGateway.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/TestExecutionSlotAllocator.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/TestExecutionSlotAllocatorFactory.java rename flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/{FailingExecutionVertexOperationsDecorator.java => TestExecutionVertexOperationsDecorator.java} (66%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorFactory.java new file mode 100644 index 0000000000..839dc078c9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorFactory.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.runtime.scheduler; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Factory for {@link DefaultExecutionSlotAllocator}. + */ +public class DefaultExecutionSlotAllocatorFactory implements ExecutionSlotAllocatorFactory { + + private final SlotProvider slotProvider; + + private final Time allocationTimeout; + + public DefaultExecutionSlotAllocatorFactory( + final SlotProvider slotProvider, + final Time allocationTimeout) { + + this.slotProvider = checkNotNull(slotProvider); + this.allocationTimeout = checkNotNull(allocationTimeout); + } + + @Override + public ExecutionSlotAllocator createInstance(final InputsLocationsRetriever inputsLocationsRetriever) { + return new DefaultExecutionSlotAllocator(slotProvider, inputsLocationsRetriever, allocationTimeout); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java index c99949870c..868cdab610 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java @@ -111,7 +111,8 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio final FailoverStrategy.Factory failoverStrategyFactory, final RestartBackoffTimeStrategy restartBackoffTimeStrategy, final ExecutionVertexOperations executionVertexOperations, - final ExecutionVertexVersioner executionVertexVersioner) throws Exception { + final ExecutionVertexVersioner executionVertexVersioner, + final ExecutionSlotAllocatorFactory executionSlotAllocatorFactory) throws Exception { super( log, @@ -140,7 +141,7 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio this.executionFailureHandler = new ExecutionFailureHandler(failoverStrategyFactory.create(getFailoverTopology()), restartBackoffTimeStrategy); this.schedulingStrategy = schedulingStrategyFactory.createInstance(this, getSchedulingTopology(), getJobGraph()); - this.executionSlotAllocator = new DefaultExecutionSlotAllocator(slotProvider, getInputsLocationsRetriever(), slotRequestTimeout); + this.executionSlotAllocator = checkNotNull(executionSlotAllocatorFactory).createInstance(getInputsLocationsRetriever()); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java index 603e14d6e7..0860c3c701 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java @@ -98,7 +98,8 @@ public class DefaultSchedulerFactory implements SchedulerNGFactory { new RestartPipelinedRegionStrategy.Factory(), restartBackoffTimeStrategy, new DefaultExecutionVertexOperations(), - new ExecutionVertexVersioner()); + new ExecutionVertexVersioner(), + new DefaultExecutionSlotAllocatorFactory(slotProvider, slotRequestTimeout)); } private SchedulingStrategyFactory createSchedulingStrategyFactory(final ScheduleMode scheduleMode) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionSlotAllocatorFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionSlotAllocatorFactory.java new file mode 100644 index 0000000000..cd763572e8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionSlotAllocatorFactory.java @@ -0,0 +1,29 @@ +/* + * 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.scheduler; + +/** + * Interface for {@link ExecutionSlotAllocator} factories. + */ +public interface ExecutionSlotAllocatorFactory { + + ExecutionSlotAllocator createInstance(InputsLocationsRetriever inputsLocationsRetriever); + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java index a3e6e0e21c..a6bc095797 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java @@ -25,7 +25,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.blob.VoidBlobWriter; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; -import org.apache.flink.runtime.clusterframework.types.SlotProfile; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.execution.ExecutionState; @@ -36,14 +35,13 @@ import org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRe import org.apache.flink.runtime.executiongraph.failover.flip1.TestRestartBackoffTimeStrategy; import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider; import org.apache.flink.runtime.io.network.partition.NoOpPartitionTracker; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; -import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; -import org.apache.flink.runtime.jobmaster.SlotRequestId; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.rest.handler.legacy.backpressure.VoidBackPressureStatsTracker; import org.apache.flink.runtime.scheduler.strategy.EagerSchedulingStrategy; @@ -59,7 +57,6 @@ import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; -import org.hamcrest.Matchers; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -73,6 +70,8 @@ import java.util.concurrent.TimeUnit; import static org.apache.flink.util.ExceptionUtils.findThrowable; import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -95,16 +94,16 @@ public class DefaultSchedulerTest extends TestLogger { private Configuration configuration; - private SubmissionTrackingTaskManagerGateway testTaskManagerGateway; - private TestRestartBackoffTimeStrategy testRestartBackoffTimeStrategy; - private FailingExecutionVertexOperationsDecorator testExecutionVertexOperations; - - private SimpleSlotProvider slotProvider; + private TestExecutionVertexOperationsDecorator testExecutionVertexOperations; private ExecutionVertexVersioner executionVertexVersioner; + private TestExecutionSlotAllocatorFactory executionSlotAllocatorFactory; + + private TestExecutionSlotAllocator testExecutionSlotAllocator; + @Before public void setUp() throws Exception { executor = Executors.newSingleThreadExecutor(); @@ -112,15 +111,15 @@ public class DefaultSchedulerTest extends TestLogger { configuration = new Configuration(); configuration.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, FailoverStrategyLoader.NO_OP_FAILOVER_STRATEGY); - testTaskManagerGateway = new SubmissionTrackingTaskManagerGateway(); testRestartBackoffTimeStrategy = new TestRestartBackoffTimeStrategy(true, 0); - testExecutionVertexOperations = new FailingExecutionVertexOperationsDecorator(new DefaultExecutionVertexOperations()); - - slotProvider = new SimpleSlotProvider(TEST_JOB_ID, 12, testTaskManagerGateway); + testExecutionVertexOperations = new TestExecutionVertexOperationsDecorator(new DefaultExecutionVertexOperations()); executionVertexVersioner = new ExecutionVertexVersioner(); + + executionSlotAllocatorFactory = new TestExecutionSlotAllocatorFactory(); + testExecutionSlotAllocator = executionSlotAllocatorFactory.getTestExecutionSlotAllocator(); } @After @@ -141,7 +140,7 @@ public class DefaultSchedulerTest extends TestLogger { createSchedulerAndStartScheduling(jobGraph); - final List deployedExecutionVertices = testTaskManagerGateway.getDeployedExecutionVertices(1, TIMEOUT_MS); + final List deployedExecutionVertices = testExecutionVertexOperations.getDeployedVertices(); final ExecutionVertexID executionVertexId = new ExecutionVertexID(onlyJobVertex.getID(), 0); assertThat(deployedExecutionVertices, contains(executionVertexId)); @@ -159,10 +158,10 @@ public class DefaultSchedulerTest extends TestLogger { testExecutionVertexOperations.disableFailDeploy(); taskRestartExecutor.triggerScheduledTasks(); - final List deployedExecutionVertices = testTaskManagerGateway.getDeployedExecutionVertices(1, TIMEOUT_MS); + final List deployedExecutionVertices = testExecutionVertexOperations.getDeployedVertices(); final ExecutionVertexID executionVertexId = new ExecutionVertexID(onlyJobVertex.getID(), 0); - assertThat(deployedExecutionVertices, contains(executionVertexId)); + assertThat(deployedExecutionVertices, contains(executionVertexId, executionVertexId)); } @Test @@ -173,7 +172,7 @@ public class DefaultSchedulerTest extends TestLogger { createSchedulerAndStartScheduling(jobGraph); - final List deployedExecutionVertices = testTaskManagerGateway.getDeployedExecutionVertices(1, TIMEOUT_MS); + final List deployedExecutionVertices = testExecutionVertexOperations.getDeployedVertices(); final ExecutionVertexID executionVertexId = new ExecutionVertexID(onlyJobVertex.getID(), 0); assertThat(deployedExecutionVertices, contains(executionVertexId)); @@ -193,7 +192,7 @@ public class DefaultSchedulerTest extends TestLogger { taskRestartExecutor.triggerScheduledTasks(); - final List deployedExecutionVertices = testTaskManagerGateway.getDeployedExecutionVertices(2, TIMEOUT_MS); + final List deployedExecutionVertices = testExecutionVertexOperations.getDeployedVertices(); final ExecutionVertexID executionVertexId = new ExecutionVertexID(onlyJobVertex.getID(), 0); assertThat(deployedExecutionVertices, contains(executionVertexId, executionVertexId)); } @@ -227,21 +226,22 @@ public class DefaultSchedulerTest extends TestLogger { waitForTermination(scheduler); final JobStatus jobStatus = scheduler.requestJobStatus(); - assertThat(jobStatus, is(Matchers.equalTo(JobStatus.FAILED))); + assertThat(jobStatus, is(equalTo(JobStatus.FAILED))); } @Test public void failJobIfNotEnoughResources() throws Exception { - drainAllAvailableSlots(); - final JobGraph jobGraph = singleNonParallelJobVertexJobGraph(); testRestartBackoffTimeStrategy.setCanRestart(false); + testExecutionSlotAllocator.disableAutoCompletePendingRequests(); final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph); + testExecutionSlotAllocator.timeoutPendingRequests(); + waitForTermination(scheduler); final JobStatus jobStatus = scheduler.requestJobStatus(); - assertThat(jobStatus, is(Matchers.equalTo(JobStatus.FAILED))); + assertThat(jobStatus, is(equalTo(JobStatus.FAILED))); Throwable failureCause = scheduler.requestJob() .getFailureInfo() @@ -252,18 +252,31 @@ public class DefaultSchedulerTest extends TestLogger { findThrowableWithMessage( failureCause, "Could not allocate the required slot within slot request timeout.").isPresent()); + assertThat(jobStatus, is(equalTo(JobStatus.FAILED))); } - private void drainAllAvailableSlots() { - final int numberOfAvailableSlots = slotProvider.getNumberOfAvailableSlots(); - for (int i = 0; i < numberOfAvailableSlots; i++) { - slotProvider.allocateSlot( - new SlotRequestId(), - new ScheduledUnit(new JobVertexID(), null, null), - SlotProfile.noRequirements(), - true, - Time.milliseconds(TIMEOUT_MS)); - } + @Test + public void skipDeploymentIfVertexVersionOutdated() { + testExecutionSlotAllocator.disableAutoCompletePendingRequests(); + + final JobGraph jobGraph = nonParallelSourceSinkJobGraph(); + final List sortedJobVertices = jobGraph.getVerticesSortedTopologicallyFromSources(); + final ExecutionVertexID sourceExecutionVertexId = new ExecutionVertexID(sortedJobVertices.get(0).getID(), 0); + final ExecutionVertexID sinkExecutionVertexId = new ExecutionVertexID(sortedJobVertices.get(1).getID(), 0); + + final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph); + testExecutionSlotAllocator.completePendingRequest(sourceExecutionVertexId); + + final ArchivedExecutionVertex sourceExecutionVertex = scheduler.requestJob().getAllExecutionVertices().iterator().next(); + final ExecutionAttemptID attemptId = sourceExecutionVertex.getCurrentExecutionAttempt().getAttemptId(); + scheduler.updateTaskExecutionState(new TaskExecutionState(jobGraph.getJobID(), attemptId, ExecutionState.FAILED)); + testRestartBackoffTimeStrategy.setCanRestart(false); + + testExecutionSlotAllocator.enableAutoCompletePendingRequests(); + taskRestartExecutor.triggerScheduledTasks(); + + assertThat(testExecutionVertexOperations.getDeployedVertices(), containsInAnyOrder(sourceExecutionVertexId, sinkExecutionVertexId)); + assertThat(scheduler.requestJob().getState(), is(equalTo(JobStatus.RUNNING))); } private void waitForTermination(final DefaultScheduler scheduler) throws Exception { @@ -279,6 +292,23 @@ public class DefaultSchedulerTest extends TestLogger { return jobGraph; } + private static JobGraph nonParallelSourceSinkJobGraph() { + final JobGraph jobGraph = new JobGraph(TEST_JOB_ID, "Testjob"); + jobGraph.setScheduleMode(ScheduleMode.EAGER); + + final JobVertex source = new JobVertex("source"); + source.setInvokableClass(NoOpInvokable.class); + jobGraph.addVertex(source); + + final JobVertex sink = new JobVertex("sink"); + sink.setInvokableClass(NoOpInvokable.class); + jobGraph.addVertex(sink); + + sink.connectNewDataSetAsInput(source, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED); + + return jobGraph; + } + private static JobVertex getOnlyJobVertex(final JobGraph jobGraph) { final List sortedVertices = jobGraph.getVerticesSortedTopologicallyFromSources(); Preconditions.checkState(sortedVertices.size() == 1); @@ -302,7 +332,7 @@ public class DefaultSchedulerTest extends TestLogger { VoidBackPressureStatsTracker.INSTANCE, executor, configuration, - slotProvider, + new SimpleSlotProvider(TEST_JOB_ID, 0), scheduledExecutorService, taskRestartExecutor, ClassLoader.getSystemClassLoader(), @@ -319,7 +349,8 @@ public class DefaultSchedulerTest extends TestLogger { new RestartPipelinedRegionStrategy.Factory(), testRestartBackoffTimeStrategy, testExecutionVertexOperations, - executionVertexVersioner); + executionVertexVersioner, + executionSlotAllocatorFactory); } private void startScheduling(final SchedulerNG scheduler) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SubmissionTrackingTaskManagerGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SubmissionTrackingTaskManagerGateway.java deleted file mode 100644 index ae1c01ba50..0000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SubmissionTrackingTaskManagerGateway.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.flink.runtime.scheduler; - -import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; -import org.apache.flink.runtime.executiongraph.TaskInformation; -import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.LinkedBlockingDeque; -import java.util.concurrent.TimeUnit; - -import static org.apache.flink.util.Preconditions.checkState; - -class SubmissionTrackingTaskManagerGateway extends SimpleAckingTaskManagerGateway { - - private final BlockingQueue taskDeploymentDescriptors = new LinkedBlockingDeque<>(); - - private boolean failSubmission; - - public void setFailSubmission(final boolean failSubmission) { - this.failSubmission = failSubmission; - } - - @Override - public CompletableFuture submitTask(final TaskDeploymentDescriptor tdd, final Time timeout) { - super.submitTask(tdd, timeout); - - taskDeploymentDescriptors.add(tdd); - - if (failSubmission) { - return FutureUtils.completedExceptionally(new RuntimeException("Task submission failed.")); - } else { - return CompletableFuture.completedFuture(Acknowledge.get()); - } - } - - public List getDeployedExecutionVertices(int num, long timeoutMs) { - final List deployedVertices = new ArrayList<>(); - for (int i = 0; i < num; i++) { - try { - final TaskDeploymentDescriptor taskDeploymentDescriptor = taskDeploymentDescriptors.poll(timeoutMs, TimeUnit.MILLISECONDS); - checkState(taskDeploymentDescriptor != null, "Expected %s tasks to be submitted within %s ms, got %s", num, timeoutMs, i); - deployedVertices.add(getExecutionVertexId(taskDeploymentDescriptor)); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - return deployedVertices; - } - - private ExecutionVertexID getExecutionVertexId(final TaskDeploymentDescriptor deploymentDescriptor) { - final TaskInformation taskInformation = deserializeTaskInformation(deploymentDescriptor); - final JobVertexID jobVertexId = taskInformation.getJobVertexId(); - final int subtaskIndex = deploymentDescriptor.getSubtaskIndex(); - return new ExecutionVertexID(jobVertexId, subtaskIndex); - } - - private TaskInformation deserializeTaskInformation(final TaskDeploymentDescriptor taskDeploymentDescriptor) { - try { - return taskDeploymentDescriptor - .getSerializedTaskInformation() - .deserializeValue(ClassLoader.getSystemClassLoader()); - } catch (IOException | ClassNotFoundException e) { - throw new RuntimeException(e); - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/TestExecutionSlotAllocator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/TestExecutionSlotAllocator.java new file mode 100644 index 0000000000..5960d7092c --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/TestExecutionSlotAllocator.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.scheduler; + +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeoutException; + +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Test {@link ExecutionSlotAllocator} implementation. + */ +public class TestExecutionSlotAllocator implements ExecutionSlotAllocator { + + private final Map pendingRequests = new HashMap<>(); + + private boolean autoCompletePendingRequests = true; + + @Override + public Collection allocateSlotsFor(final Collection schedulingRequirementsCollection) { + final List slotVertexAssignments = createSlotVertexAssignments(schedulingRequirementsCollection); + registerPendingRequests(slotVertexAssignments); + maybeCompletePendingRequests(); + return slotVertexAssignments; + } + + private void registerPendingRequests(final List slotVertexAssignments) { + for (SlotExecutionVertexAssignment slotVertexAssignment : slotVertexAssignments) { + pendingRequests.put(slotVertexAssignment.getExecutionVertexId(), slotVertexAssignment); + } + } + + private List createSlotVertexAssignments( + final Collection schedulingRequirementsCollection) { + + final List result = new ArrayList<>(); + for (ExecutionVertexSchedulingRequirements schedulingRequirements : schedulingRequirementsCollection) { + final ExecutionVertexID executionVertexId = schedulingRequirements.getExecutionVertexId(); + final CompletableFuture logicalSlotFuture = new CompletableFuture<>(); + result.add(new SlotExecutionVertexAssignment(executionVertexId, logicalSlotFuture)); + } + return result; + } + + private void maybeCompletePendingRequests() { + if (autoCompletePendingRequests) { + completePendingRequests(); + } + } + + public void completePendingRequests() { + final Collection vertexIds = new ArrayList<>(pendingRequests.keySet()); + vertexIds.forEach(this::completePendingRequest); + } + + public void completePendingRequest(final ExecutionVertexID executionVertexId) { + final SlotExecutionVertexAssignment slotVertexAssignment = removePendingRequest(executionVertexId); + checkState(slotVertexAssignment != null); + slotVertexAssignment + .getLogicalSlotFuture() + .complete(new TestingLogicalSlotBuilder().createTestingLogicalSlot()); + } + + private SlotExecutionVertexAssignment removePendingRequest(final ExecutionVertexID executionVertexId) { + return pendingRequests.remove(executionVertexId); + } + + public void timeoutPendingRequests() { + final Collection vertexIds = new ArrayList<>(pendingRequests.keySet()); + vertexIds.forEach(this::timeoutPendingRequest); + } + + public void timeoutPendingRequest(final ExecutionVertexID executionVertexId) { + final SlotExecutionVertexAssignment slotVertexAssignment = removePendingRequest(executionVertexId); + checkState(slotVertexAssignment != null); + slotVertexAssignment + .getLogicalSlotFuture() + .completeExceptionally(new TimeoutException()); + } + + public void enableAutoCompletePendingRequests() { + autoCompletePendingRequests = true; + } + + public void disableAutoCompletePendingRequests() { + autoCompletePendingRequests = false; + } + + @Override + public void cancel(final ExecutionVertexID executionVertexId) { + final SlotExecutionVertexAssignment slotVertexAssignment = removePendingRequest(executionVertexId); + if (slotVertexAssignment != null) { + slotVertexAssignment + .getLogicalSlotFuture() + .cancel(false); + } + } + + @Override + public CompletableFuture stop() { + return CompletableFuture.completedFuture(null); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/TestExecutionSlotAllocatorFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/TestExecutionSlotAllocatorFactory.java new file mode 100644 index 0000000000..e5d451ada8 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/TestExecutionSlotAllocatorFactory.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.runtime.scheduler; + +/** + * Factory for {@link TestExecutionSlotAllocatorFactory}. + */ +public class TestExecutionSlotAllocatorFactory implements ExecutionSlotAllocatorFactory { + + private final TestExecutionSlotAllocator testExecutionSlotAllocator = new TestExecutionSlotAllocator(); + + @Override + public ExecutionSlotAllocator createInstance(final InputsLocationsRetriever ignored) { + return testExecutionSlotAllocator; + } + + public TestExecutionSlotAllocator getTestExecutionSlotAllocator() { + return testExecutionSlotAllocator; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/FailingExecutionVertexOperationsDecorator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/TestExecutionVertexOperationsDecorator.java similarity index 66% rename from flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/FailingExecutionVertexOperationsDecorator.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/TestExecutionVertexOperationsDecorator.java index 76db6d6636..5045751940 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/FailingExecutionVertexOperationsDecorator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/TestExecutionVertexOperationsDecorator.java @@ -21,36 +21,45 @@ package org.apache.flink.runtime.scheduler; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Allows to fail ExecutionVertex operations for testing. + * {@link ExecutionVertexOperations} decorator that enables instrumentation of execution vertex + * operations for testing purposes. */ -public class FailingExecutionVertexOperationsDecorator implements ExecutionVertexOperations { +public class TestExecutionVertexOperationsDecorator implements ExecutionVertexOperations { private final ExecutionVertexOperations delegate; + private final List deployedVertices = new ArrayList<>(); + private boolean failDeploy; - public FailingExecutionVertexOperationsDecorator(final ExecutionVertexOperations delegate) { + public TestExecutionVertexOperationsDecorator(final ExecutionVertexOperations delegate) { this.delegate = checkNotNull(delegate); } @Override public void deploy(final ExecutionVertex executionVertex) throws JobException { + deployedVertices.add(executionVertex.getID()); + if (failDeploy) { throw new RuntimeException("Expected"); - } else { - delegate.deploy(executionVertex); } + + delegate.deploy(executionVertex); } @Override public CompletableFuture cancel(final ExecutionVertex executionVertex) { - return delegate.cancel(executionVertex); + return delegate.cancel(executionVertex); } public void enableFailDeploy() { @@ -61,4 +70,7 @@ public class FailingExecutionVertexOperationsDecorator implements ExecutionVerte failDeploy = false; } + public List getDeployedVertices() { + return Collections.unmodifiableList(deployedVertices); + } } -- Gitee From 2f1200623f465b8e35daa86613eee676cac65b2b Mon Sep 17 00:00:00 2001 From: xuyang1706 Date: Wed, 31 Jul 2019 20:46:45 +0800 Subject: [PATCH 195/268] [FLINK-13339][ml] Add an implementation of Flink ML Pipeline api. - Add an abstract implemention of PipelineStage, Estimator, Transformer, Model. - Add MLEnvironment to hold the execution environment and others environment shared variable. - Add AlgoOperator for the implementation of algorithms. - Add BatchOperator and StreamOperator based on AlgoOperator. - Add TableSourceBatchOp and TableSourceStreamOp. --- flink-ml-parent/flink-ml-lib/pom.xml | 19 +- .../apache/flink/ml/common/MLEnvironment.java | 158 ++++++++++++++ .../flink/ml/common/MLEnvironmentFactory.java | 119 +++++++++++ .../flink/ml/operator/AlgoOperator.java | 199 ++++++++++++++++++ .../ml/operator/batch/BatchOperator.java | 118 +++++++++++ .../batch/source/TableSourceBatchOp.java | 42 ++++ .../ml/operator/stream/StreamOperator.java | 121 +++++++++++ .../stream/source/TableSourceStreamOp.java | 41 ++++ .../ml/params/shared/HasMLEnvironmentId.java | 45 ++++ .../flink/ml/pipeline/EstimatorBase.java | 103 +++++++++ .../apache/flink/ml/pipeline/ModelBase.java | 68 ++++++ .../flink/ml/pipeline/PipelineStageBase.java | 72 +++++++ .../flink/ml/pipeline/TransformerBase.java | 100 +++++++++ .../flink/ml/common/MLEnvironmentTest.java | 74 +++++++ .../flink/ml/pipeline/EstimatorBaseTest.java | 93 ++++++++ .../ml/pipeline/PipelineStageTestBase.java | 67 ++++++ .../ml/pipeline/TransformerBaseTest.java | 92 ++++++++ 17 files changed, 1530 insertions(+), 1 deletion(-) create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/MLEnvironment.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/MLEnvironmentFactory.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/AlgoOperator.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/batch/BatchOperator.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/batch/source/TableSourceBatchOp.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/stream/StreamOperator.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/stream/source/TableSourceStreamOp.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/params/shared/HasMLEnvironmentId.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/EstimatorBase.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/ModelBase.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/PipelineStageBase.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/TransformerBase.java create mode 100644 flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/common/MLEnvironmentTest.java create mode 100644 flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/EstimatorBaseTest.java create mode 100644 flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/PipelineStageTestBase.java create mode 100644 flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/TransformerBaseTest.java diff --git a/flink-ml-parent/flink-ml-lib/pom.xml b/flink-ml-parent/flink-ml-lib/pom.xml index 391b2acd86..5cd4793ccd 100644 --- a/flink-ml-parent/flink-ml-lib/pom.xml +++ b/flink-ml-parent/flink-ml-lib/pom.xml @@ -26,7 +26,8 @@ under the License. 1.10-SNAPSHOT - flink-ml-lib + flink-ml-lib_${scala.binary.version} + flink-ml-lib @@ -34,6 +35,22 @@ under the License. flink-ml-api ${project.version} + + org.apache.flink + flink-table-api-java + ${project.version} + + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${project.version} + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test + com.github.fommil.netlib core diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/MLEnvironment.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/MLEnvironment.java new file mode 100644 index 0000000000..f9deceaec6 --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/MLEnvironment.java @@ -0,0 +1,158 @@ +/* + * 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.ml.common; + +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.java.BatchTableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; + +/** + * The MLEnvironment stores the necessary context in Flink. + * Each MLEnvironment will be associated with a unique ID. + * The operations associated with the same MLEnvironment ID + * will share the same Flink job context. + * + *

Both MLEnvironment ID and MLEnvironment can only be retrieved from MLEnvironmentFactory. + * + * @see ExecutionEnvironment + * @see StreamExecutionEnvironment + * @see BatchTableEnvironment + * @see StreamTableEnvironment + */ +public class MLEnvironment { + private ExecutionEnvironment env; + private StreamExecutionEnvironment streamEnv; + private BatchTableEnvironment batchTableEnv; + private StreamTableEnvironment streamTableEnv; + + /** + * Construct with null that the class can load the environment in the `get` method. + */ + public MLEnvironment() { + this(null, null, null, null); + } + + /** + * Construct with the batch environment and the the batch table environment given by user. + * + *

The env can be null which will be loaded in the `get` method. + * + * @param batchEnv the ExecutionEnvironment + * @param batchTableEnv the BatchTableEnvironment + */ + public MLEnvironment( + ExecutionEnvironment batchEnv, + BatchTableEnvironment batchTableEnv) { + this(batchEnv, batchTableEnv, null, null); + } + + /** + * Construct with the stream environment and the the stream table environment given by user. + * + *

The env can be null which will be loaded in the `get` method. + * + * @param streamEnv the StreamExecutionEnvironment + * @param streamTableEnv the StreamTableEnvironment + */ + public MLEnvironment( + StreamExecutionEnvironment streamEnv, + StreamTableEnvironment streamTableEnv) { + this(null, null, streamEnv, streamTableEnv); + } + + /** + * Construct with env given by user. + * + *

The env can be null which will be loaded in the `get` method. + * + * @param batchEnv the ExecutionEnvironment + * @param batchTableEnv the BatchTableEnvironment + * @param streamEnv the StreamExecutionEnvironment + * @param streamTableEnv the StreamTableEnvironment + */ + public MLEnvironment( + ExecutionEnvironment batchEnv, + BatchTableEnvironment batchTableEnv, + StreamExecutionEnvironment streamEnv, + StreamTableEnvironment streamTableEnv) { + this.env = batchEnv; + this.batchTableEnv = batchTableEnv; + this.streamEnv = streamEnv; + this.streamTableEnv = streamTableEnv; + } + + /** + * Get the ExecutionEnvironment. + * if the ExecutionEnvironment has not been set, it initial the ExecutionEnvironment + * with default Configuration. + * + * @return the batch {@link ExecutionEnvironment} + */ + public ExecutionEnvironment getExecutionEnvironment() { + if (null == env) { + env = ExecutionEnvironment.getExecutionEnvironment(); + } + return env; + } + + /** + * Get the StreamExecutionEnvironment. + * if the StreamExecutionEnvironment has not been set, it initial the StreamExecutionEnvironment + * with default Configuration. + * + * @return the {@link StreamExecutionEnvironment} + */ + public StreamExecutionEnvironment getStreamExecutionEnvironment() { + if (null == streamEnv) { + streamEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + } + return streamEnv; + } + + /** + * Get the BatchTableEnvironment. + * if the BatchTableEnvironment has not been set, it initial the BatchTableEnvironment + * with default Configuration. + * + * @return the {@link BatchTableEnvironment} + */ + public BatchTableEnvironment getBatchTableEnvironment() { + if (null == batchTableEnv) { + batchTableEnv = BatchTableEnvironment.create(getExecutionEnvironment()); + } + return batchTableEnv; + } + + /** + * Get the StreamTableEnvironment. + * if the StreamTableEnvironment has not been set, it initial the StreamTableEnvironment + * with default Configuration. + * + * @return the {@link StreamTableEnvironment} + */ + public StreamTableEnvironment getStreamTableEnvironment() { + if (null == streamTableEnv) { + streamTableEnv = StreamTableEnvironment.create(getStreamExecutionEnvironment()); + } + return streamTableEnv; + } +} + diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/MLEnvironmentFactory.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/MLEnvironmentFactory.java new file mode 100644 index 0000000000..c1b298632d --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/MLEnvironmentFactory.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.ml.common; + +import org.apache.flink.util.Preconditions; + +import java.util.HashMap; + +/** + * Factory to get the MLEnvironment using a MLEnvironmentId. + * + *

The following code snippet shows how to interact with MLEnvironmentFactory. + *

+ * {@code
+ * long mlEnvId = MLEnvironmentFactory.getNewMLEnvironmentId();
+ * MLEnvironment mlEnv = MLEnvironmentFactory.get(mlEnvId);
+ * }
+ * 
+ */ +public class MLEnvironmentFactory { + + /** + * The default MLEnvironmentId. + */ + public static final Long DEFAULT_ML_ENVIRONMENT_ID = 0L; + + /** + * A monotonically increasing id for the MLEnvironments. + * Each id uniquely identifies an MLEnvironment. + */ + private static Long nextId = 1L; + + /** + * Map that hold the MLEnvironment and use the MLEnvironmentId as its key. + */ + private static final HashMap map = new HashMap<>(); + + static { + map.put(DEFAULT_ML_ENVIRONMENT_ID, new MLEnvironment()); + } + + /** + * Get the MLEnvironment using a MLEnvironmentId. + * + * @param mlEnvId the MLEnvironmentId + * @return the MLEnvironment + */ + public static synchronized MLEnvironment get(Long mlEnvId) { + if (!map.containsKey(mlEnvId)) { + throw new IllegalArgumentException( + String.format("Cannot find MLEnvironment for MLEnvironmentId %s." + + " Did you get the MLEnvironmentId by calling getNewMLEnvironmentId?", mlEnvId)); + } + + return map.get(mlEnvId); + } + + /** + * Get the MLEnvironment use the default MLEnvironmentId. + * + * @return the default MLEnvironment. + */ + public static synchronized MLEnvironment getDefault() { + return get(DEFAULT_ML_ENVIRONMENT_ID); + } + + /** + * Create a unique MLEnvironment id and register a new MLEnvironment in the factory. + * + * @return the MLEnvironment id. + */ + public static synchronized Long getNewMLEnvironmentId() { + return registerMLEnvironment(new MLEnvironment()); + } + + /** + * Register a new MLEnvironment to the factory and return a new MLEnvironment id. + * + * @param env the MLEnvironment that will be stored in the factory. + * @return the MLEnvironment id. + */ + public static synchronized Long registerMLEnvironment(MLEnvironment env) { + map.put(nextId, env); + return nextId++; + } + + /** + * Remove the MLEnvironment using the MLEnvironmentId. + * + * @param mlEnvId the id. + * @return the removed MLEnvironment + */ + public static synchronized MLEnvironment remove(Long mlEnvId) { + Preconditions.checkNotNull(mlEnvId, "The environment id cannot be null."); + // Never remove the default MLEnvironment. Just return the default environment. + if (DEFAULT_ML_ENVIRONMENT_ID.equals(mlEnvId)) { + return getDefault(); + } else { + return map.remove(mlEnvId); + } + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/AlgoOperator.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/AlgoOperator.java new file mode 100644 index 0000000000..43c9a89cad --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/AlgoOperator.java @@ -0,0 +1,199 @@ +/* + * 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.ml.operator; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.ml.api.misc.param.Params; +import org.apache.flink.ml.api.misc.param.WithParams; +import org.apache.flink.ml.params.shared.HasMLEnvironmentId; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; + +/** + * Base class for algorithm operators. + * + *

Base class for the algorithm operators. It hosts the parameters and output + * tables of an algorithm operator. Each AlgoOperator may have one or more output tables. + * One of the output table is the primary output table which can be obtained by calling + * {@link #getOutput}. The other output tables are side output tables that can be obtained + * by calling {@link #getSideOutputs()}. + * + *

The input of an AlgoOperator is defined in the subclasses of the AlgoOperator. + * + * @param The class type of the {@link AlgoOperator} implementation itself + */ +public abstract class AlgoOperator> + implements WithParams, HasMLEnvironmentId, Serializable { + + /** + * Params for algorithms. + */ + private Params params; + + /** + * The table held by operator. + */ + private Table output = null; + + /** + * The side outputs of operator that be similar to the stream's side outputs. + */ + private Table[] sideOutputs = null; + + /** + * Construct the operator with empty Params. + * + *

This constructor is especially useful when users want to set parameters + * for the algorithm operators. For example: + * SplitBatchOp is widely used in ML data pre-processing, + * which splits one dataset into two dataset: training set and validation set. + * It is very convenient for us to write code like this: + *

+	 * {@code
+	 * new SplitBatchOp().setSplitRatio(0.9)
+	 * }
+	 * 
+ */ + protected AlgoOperator() { + this(null); + } + + /** + * Construct the operator with the initial Params. + */ + protected AlgoOperator(Params params) { + if (null == params) { + this.params = new Params(); + } else { + this.params = params.clone(); + } + } + + @Override + public Params getParams() { + return this.params; + } + + /** + * Returns the table held by operator. + */ + public Table getOutput() { + return this.output; + } + + /** + * Returns the side outputs. + */ + public Table[] getSideOutputs() { + return this.sideOutputs; + } + + /** + * Set the side outputs. + * + * @param sideOutputs the side outputs set the operator. + */ + protected void setSideOutputs(Table[] sideOutputs) { + this.sideOutputs = sideOutputs; + } + + /** + * Set the table held by operator. + * + * @param output the output table. + */ + protected void setOutput(Table output) { + this.output = output; + } + + /** + * Returns the column names of the output table. + */ + public String[] getColNames() { + return getSchema().getFieldNames(); + } + + /** + * Returns the column types of the output table. + */ + public TypeInformation[] getColTypes() { + return getSchema().getFieldTypes(); + } + + /** + * Get the column names of the specified side-output table. + * + * @param index the index of the table. + * @return the column types of the table. + */ + public String[] getSideOutputColNames(int index) { + checkSideOutputAccessibility(index); + + return sideOutputs[index].getSchema().getFieldNames(); + } + + /** + * Get the column types of the specified side-output table. + * + * @param index the index of the table. + * @return the column types of the table. + */ + public TypeInformation[] getSideOutputColTypes(int index) { + checkSideOutputAccessibility(index); + + return sideOutputs[index].getSchema().getFieldTypes(); + } + + /** + * Returns the schema of the output table. + */ + public TableSchema getSchema() { + return this.getOutput().getSchema(); + } + + @Override + public String toString() { + return getOutput().toString(); + } + + protected static void checkOpSize(int size, AlgoOperator... inputs) { + Preconditions.checkNotNull(inputs, "Operators should not be null."); + Preconditions.checkState(inputs.length == size, "The size of operators should be equal to " + + size + ", current: " + inputs.length); + } + + protected static void checkMinOpSize(int size, AlgoOperator... inputs) { + Preconditions.checkNotNull(inputs, "Operators should not be null."); + Preconditions.checkState(inputs.length >= size, "The size of operators should be equal or greater than " + + size + ", current: " + inputs.length); + } + + private void checkSideOutputAccessibility(int index) { + Preconditions.checkNotNull(sideOutputs, + "There is not side-outputs in this AlgoOperator."); + Preconditions.checkState(index >= 0 && index < sideOutputs.length, + String.format("The index(%s) of side-outputs is out of bound.", index)); + Preconditions.checkNotNull(sideOutputs[index], + String.format("The %snd of side-outputs is null. Maybe the operator has not been linked.", index)); + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/batch/BatchOperator.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/batch/BatchOperator.java new file mode 100644 index 0000000000..d655fe8f2a --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/batch/BatchOperator.java @@ -0,0 +1,118 @@ +/* + * 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.ml.operator.batch; + +import org.apache.flink.ml.api.misc.param.Params; +import org.apache.flink.ml.operator.AlgoOperator; +import org.apache.flink.ml.operator.batch.source.TableSourceBatchOp; +import org.apache.flink.table.api.Table; + +/** + * Base class of batch algorithm operators. + * + *

This class extends {@link AlgoOperator} to support data transmission between BatchOperators. + */ +public abstract class BatchOperator> extends AlgoOperator { + + public BatchOperator() { + super(); + } + + /** + * The constructor of BatchOperator with {@link Params}. + * @param params the initial Params. + */ + public BatchOperator(Params params) { + super(params); + } + + /** + * Link to another {@link BatchOperator}. + * + *

Link the next BatchOperator using this BatchOperator as its input. + * + *

For example: + * + *

+	 * {@code
+	 * BatchOperator a = ...;
+	 * BatchOperator b = ...;
+	 * BatchOperator c = a.link(b)
+	 * }
+	 * 
+ * + *

The BatchOperator c in the above code + * is the same instance as b which takes + * a as its input. + * Note that BatchOperator b will be changed + * to link from BatchOperator a. + * + * @param next The operator that will be modified to add this operator to its input. + * @param type of BatchOperator returned + * @return the linked next + * @see #linkFrom(BatchOperator[]) + */ + public > B link(B next) { + next.linkFrom(this); + return next; + } + + /** + * Link from others {@link BatchOperator}. + * + *

Link this object to BatchOperator using the BatchOperators as its input. + * + *

For example: + * + *

+	 * {@code
+	 * BatchOperator a = ...;
+	 * BatchOperator b = ...;
+	 * BatchOperator c = ...;
+	 *
+	 * BatchOperator d = c.linkFrom(a, b)
+	 * }
+	 * 
+ * + *

The d in the above code is the same + * instance as BatchOperator c which takes + * both a and b as its input. + * + *

note: It is not recommended to linkFrom itself or linkFrom the same group inputs twice. + * + * @param inputs the linked inputs + * @return the linked this object + */ + public abstract T linkFrom(BatchOperator... inputs); + + /** + * create a new BatchOperator from table. + * @param table the input table + * @return the new BatchOperator + */ + public static BatchOperator fromTable(Table table) { + return new TableSourceBatchOp(table); + } + + protected static BatchOperator checkAndGetFirst(BatchOperator ... inputs) { + checkOpSize(1, inputs); + return inputs[0]; + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/batch/source/TableSourceBatchOp.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/batch/source/TableSourceBatchOp.java new file mode 100644 index 0000000000..7f8d6af5df --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/batch/source/TableSourceBatchOp.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.ml.operator.batch.source; + +import org.apache.flink.ml.operator.batch.BatchOperator; +import org.apache.flink.table.api.Table; +import org.apache.flink.util.Preconditions; + +/** + * Transform the Table to SourceBatchOp. + */ +public final class TableSourceBatchOp extends BatchOperator { + + public TableSourceBatchOp(Table table) { + super(null); + Preconditions.checkArgument(table != null, "The source table cannot be null."); + this.setOutput(table); + } + + @Override + public TableSourceBatchOp linkFrom(BatchOperator... inputs) { + throw new UnsupportedOperationException("Table source operator should not have any upstream to link from."); + } + +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/stream/StreamOperator.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/stream/StreamOperator.java new file mode 100644 index 0000000000..870623afae --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/stream/StreamOperator.java @@ -0,0 +1,121 @@ +/* + * 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.ml.operator.stream; + +import org.apache.flink.ml.api.misc.param.Params; +import org.apache.flink.ml.operator.AlgoOperator; +import org.apache.flink.ml.operator.stream.source.TableSourceStreamOp; +import org.apache.flink.table.api.Table; + +/** + * Base class of stream algorithm operators. + * + *

This class extends {@link AlgoOperator} to support data transmission between StreamOperator. + */ +public abstract class StreamOperator> extends AlgoOperator { + + public StreamOperator() { + super(); + } + + /** + * The constructor of StreamOperator with {@link Params}. + * + * @param params the initial Params. + */ + public StreamOperator(Params params) { + super(params); + } + + /** + * Link to another {@link StreamOperator}. + * + *

Link the next StreamOperator using this StreamOperator as its input. + * + *

For example: + * + *

+	 * {@code
+	 * StreamOperator a = ...;
+	 * StreamOperator b = ...;
+	 *
+	 * StreamOperator c = a.link(b)
+	 * }
+	 * 
+ * + *

The StreamOperator c in the above code + * is the same instance as b which takes + * a as its input. + * Note that StreamOperator b will be changed + * to link from StreamOperator a. + * + * @param next the linked StreamOperator + * @param type of StreamOperator returned + * @return the linked next + * @see #linkFrom(StreamOperator[]) + */ + public > S link(S next) { + next.linkFrom(this); + return next; + } + + /** + * Link from others {@link StreamOperator}. + * + *

Link this object to StreamOperator using the StreamOperators as its input. + * + *

For example: + * + *

+	 * {@code
+	 * StreamOperator a = ...;
+	 * StreamOperator b = ...;
+	 * StreamOperator c = ...;
+	 *
+	 * StreamOperator d = c.linkFrom(a, b)
+	 * }
+	 * 
+ * + *

The d in the above code is the same + * instance as StreamOperator c which takes + * both a and b as its input. + * + *

note: It is not recommended to linkFrom itself or linkFrom the same group inputs twice. + * + * @param inputs the linked inputs + * @return the linked this object + */ + public abstract T linkFrom(StreamOperator... inputs); + + /** + * create a new StreamOperator from table. + * + * @param table the input table + * @return the new StreamOperator + */ + public static StreamOperator fromTable(Table table) { + return new TableSourceStreamOp(table); + } + + protected static StreamOperator checkAndGetFirst(StreamOperator... inputs) { + checkOpSize(1, inputs); + return inputs[0]; + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/stream/source/TableSourceStreamOp.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/stream/source/TableSourceStreamOp.java new file mode 100644 index 0000000000..8ea2f02c15 --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/operator/stream/source/TableSourceStreamOp.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.ml.operator.stream.source; + +import org.apache.flink.ml.operator.stream.StreamOperator; +import org.apache.flink.table.api.Table; +import org.apache.flink.util.Preconditions; + +/** + * Transform the Table to SourceStreamOp. + */ +public final class TableSourceStreamOp extends StreamOperator { + + public TableSourceStreamOp(Table table) { + super(null); + Preconditions.checkArgument(table != null, "The source table cannot be null."); + this.setOutput(table); + } + + @Override + public TableSourceStreamOp linkFrom(StreamOperator... inputs) { + throw new UnsupportedOperationException("Table source operator should not have any upstream to link from."); + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/params/shared/HasMLEnvironmentId.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/params/shared/HasMLEnvironmentId.java new file mode 100644 index 0000000000..ae7c7fe198 --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/params/shared/HasMLEnvironmentId.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.ml.params.shared; + +import org.apache.flink.ml.api.misc.param.ParamInfo; +import org.apache.flink.ml.api.misc.param.ParamInfoFactory; +import org.apache.flink.ml.api.misc.param.WithParams; +import org.apache.flink.ml.common.MLEnvironmentFactory; + +/** + * An interface for classes with a parameter specifying the id of MLEnvironment. + */ +public interface HasMLEnvironmentId extends WithParams { + + ParamInfo ML_ENVIRONMENT_ID = ParamInfoFactory + .createParamInfo("MLEnvironmentId", Long.class) + .setDescription("ID of ML environment.") + .setHasDefaultValue(MLEnvironmentFactory.DEFAULT_ML_ENVIRONMENT_ID) + .build(); + + default Long getMLEnvironmentId() { + return get(ML_ENVIRONMENT_ID); + } + + default T setMLEnvironmentId(Long value) { + return set(ML_ENVIRONMENT_ID, value); + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/EstimatorBase.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/EstimatorBase.java new file mode 100644 index 0000000000..3b6bcc143d --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/EstimatorBase.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.ml.pipeline; + +import org.apache.flink.ml.api.core.Estimator; +import org.apache.flink.ml.api.misc.param.Params; +import org.apache.flink.ml.operator.batch.BatchOperator; +import org.apache.flink.ml.operator.batch.source.TableSourceBatchOp; +import org.apache.flink.ml.operator.stream.StreamOperator; +import org.apache.flink.ml.operator.stream.source.TableSourceStreamOp; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.TableImpl; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.util.Preconditions; + +/** + * The base class for estimator implementations. + * + * @param A subclass of the {@link EstimatorBase}, used by + * {@link org.apache.flink.ml.api.misc.param.WithParams} + * @param class type of the {@link ModelBase} this Estimator produces. + */ +public abstract class EstimatorBase, M extends ModelBase> + extends PipelineStageBase implements Estimator { + + public EstimatorBase() { + super(); + } + + public EstimatorBase(Params params) { + super(params); + } + + @Override + public M fit(TableEnvironment tEnv, Table input) { + Preconditions.checkArgument(input != null, "Input CAN NOT BE null!"); + Preconditions.checkArgument( + tableEnvOf(input) == tEnv, + "The input table is not in the specified table environment."); + return fit(input); + } + + /** + * Train and produce a {@link ModelBase} which fits the records in the given {@link Table}. + * + * @param input the table with records to train the Model. + * @return a model trained to fit on the given Table. + */ + public M fit(Table input) { + Preconditions.checkArgument(input != null, "Input CAN NOT BE null!"); + if (((TableImpl) input).getTableEnvironment() instanceof StreamTableEnvironment) { + TableSourceStreamOp source = new TableSourceStreamOp(input); + if (this.params.contains(ML_ENVIRONMENT_ID)) { + source.setMLEnvironmentId(this.params.get(ML_ENVIRONMENT_ID)); + } + return fit(source); + } else { + TableSourceBatchOp source = new TableSourceBatchOp(input); + if (this.params.contains(ML_ENVIRONMENT_ID)) { + source.setMLEnvironmentId(this.params.get(ML_ENVIRONMENT_ID)); + } + return fit(source); + } + } + + /** + * Train and produce a {@link ModelBase} which fits the records from the given {@link BatchOperator}. + * + * @param input the table with records to train the Model. + * @return a model trained to fit on the given Table. + */ + protected abstract M fit(BatchOperator input); + + /** + * Online learning and produce {@link ModelBase} series which fit the streaming records from the given {@link + * StreamOperator}. + * + * @param input the StreamOperator with streaming records to online train the Model series. + * @return the model series trained to fit on the streaming data from given StreamOperator. + */ + protected M fit(StreamOperator input) { + throw new UnsupportedOperationException("NOT supported yet!"); + } + +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/ModelBase.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/ModelBase.java new file mode 100644 index 0000000000..98f007fd4e --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/ModelBase.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.ml.pipeline; + +import org.apache.flink.ml.api.core.Model; +import org.apache.flink.ml.api.misc.param.Params; +import org.apache.flink.table.api.Table; + +/** + * The base class for a machine learning model. + * + * @param The class type of the {@link ModelBase} implementation itself + */ +public abstract class ModelBase> extends TransformerBase + implements Model { + + protected Table modelData; + + public ModelBase() { + super(); + } + + public ModelBase(Params params) { + super(params); + } + + /** + * Get model data as Table representation. + * + * @return the Table + */ + public Table getModelData() { + return this.modelData; + } + + /** + * Set the model data using the Table. + * + * @param modelData the Table. + * @return {@link ModelBase} itself + */ + public M setModelData(Table modelData) { + this.modelData = modelData; + return (M) this; + } + + @Override + public M clone() throws CloneNotSupportedException { + return (M) super.clone().setModelData(this.modelData); + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/PipelineStageBase.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/PipelineStageBase.java new file mode 100644 index 0000000000..cb932569c7 --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/PipelineStageBase.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.ml.pipeline; + +import org.apache.flink.ml.api.misc.param.Params; +import org.apache.flink.ml.api.misc.param.WithParams; +import org.apache.flink.ml.params.shared.HasMLEnvironmentId; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.TableImpl; + +/** + * The base class for a stage in a pipeline, either an [[EstimatorBase]] or a [[TransformerBase]]. + * + *

The PipelineStageBase maintains the parameters for the stage. + * A default constructor is needed in order to restore a pipeline stage. + * + * @param The class type of the {@link PipelineStageBase} implementation itself, used by {@link + * org.apache.flink.ml.api.misc.param.WithParams} and Cloneable. + */ +public abstract class PipelineStageBase> + implements WithParams , HasMLEnvironmentId, Cloneable { + protected Params params; + + public PipelineStageBase() { + this(null); + } + + public PipelineStageBase(Params params) { + if (null == params) { + this.params = new Params(); + } else { + this.params = params.clone(); + } + } + + @Override + public Params getParams() { + if (null == this.params) { + this.params = new Params(); + } + return this.params; + } + + @Override + public S clone() throws CloneNotSupportedException { + PipelineStageBase result = (PipelineStageBase) super.clone(); + result.params = this.params.clone(); + return (S) result; + } + + protected static TableEnvironment tableEnvOf(Table table) { + return ((TableImpl) table).getTableEnvironment(); + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/TransformerBase.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/TransformerBase.java new file mode 100644 index 0000000000..ed3c374cbd --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/pipeline/TransformerBase.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.ml.pipeline; + +import org.apache.flink.ml.api.core.Transformer; +import org.apache.flink.ml.api.misc.param.Params; +import org.apache.flink.ml.operator.batch.BatchOperator; +import org.apache.flink.ml.operator.batch.source.TableSourceBatchOp; +import org.apache.flink.ml.operator.stream.StreamOperator; +import org.apache.flink.ml.operator.stream.source.TableSourceStreamOp; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.util.Preconditions; + +/** + * The base class for transformer implementations. + * + * @param A subclass of {@link TransformerBase}, used by {@link + * org.apache.flink.ml.api.misc.param.WithParams} + */ +public abstract class TransformerBase> + extends PipelineStageBase implements Transformer { + + public TransformerBase() { + super(); + } + + public TransformerBase(Params params) { + super(params); + } + + @Override + public Table transform(TableEnvironment tEnv, Table input) { + Preconditions.checkArgument(input != null, "Input CAN NOT BE null!"); + Preconditions.checkArgument( + tableEnvOf(input) == tEnv, + "The input table is not in the specified table environment."); + return transform(input); + } + + /** + * Applies the transformer on the input table, and returns the result table. + * + * @param input the table to be transformed + * @return the transformed table + */ + public Table transform(Table input) { + Preconditions.checkArgument(input != null, "Input CAN NOT BE null!"); + if (tableEnvOf(input) instanceof StreamTableEnvironment) { + TableSourceStreamOp source = new TableSourceStreamOp(input); + if (this.params.contains(ML_ENVIRONMENT_ID)) { + source.setMLEnvironmentId(this.params.get(ML_ENVIRONMENT_ID)); + } + return transform(source).getOutput(); + } else { + TableSourceBatchOp source = new TableSourceBatchOp(input); + if (this.params.contains(ML_ENVIRONMENT_ID)) { + source.setMLEnvironmentId(this.params.get(ML_ENVIRONMENT_ID)); + } + return transform(source).getOutput(); + } + } + + /** + * Applies the transformer on the input batch data from BatchOperator, and returns the batch result data with + * BatchOperator. + * + * @param input the input batch data from BatchOperator + * @return the transformed batch result data + */ + protected abstract BatchOperator transform(BatchOperator input); + + /** + * Applies the transformer on the input streaming data from StreamOperator, and returns the streaming result data + * with StreamOperator. + * + * @param input the input streaming data from StreamOperator + * @return the transformed streaming result data + */ + protected abstract StreamOperator transform(StreamOperator input); + +} diff --git a/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/common/MLEnvironmentTest.java b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/common/MLEnvironmentTest.java new file mode 100644 index 0000000000..60b18fe595 --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/common/MLEnvironmentTest.java @@ -0,0 +1,74 @@ +/* + * 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.ml.common; + +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.java.BatchTableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; + +import org.junit.Assert; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** + * Test cases for MLEnvironment. + */ +public class MLEnvironmentTest { + @Test + public void testDefaultConstructor() { + MLEnvironment mlEnvironment = new MLEnvironment(); + Assert.assertNotNull(mlEnvironment.getExecutionEnvironment()); + Assert.assertNotNull(mlEnvironment.getBatchTableEnvironment()); + Assert.assertNotNull(mlEnvironment.getStreamExecutionEnvironment()); + Assert.assertNotNull(mlEnvironment.getStreamTableEnvironment()); + } + + @Test + public void testConstructWithBatchEnv() { + ExecutionEnvironment executionEnvironment = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment batchTableEnvironment = BatchTableEnvironment.create(executionEnvironment); + + MLEnvironment mlEnvironment = new MLEnvironment(executionEnvironment, batchTableEnvironment); + + Assert.assertSame(mlEnvironment.getExecutionEnvironment(), executionEnvironment); + Assert.assertSame(mlEnvironment.getBatchTableEnvironment(), batchTableEnvironment); + } + + @Test + public void testConstructWithStreamEnv() { + StreamExecutionEnvironment streamExecutionEnvironment = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment streamTableEnvironment = StreamTableEnvironment.create(streamExecutionEnvironment); + + MLEnvironment mlEnvironment = new MLEnvironment(streamExecutionEnvironment, streamTableEnvironment); + + Assert.assertSame(mlEnvironment.getStreamExecutionEnvironment(), streamExecutionEnvironment); + Assert.assertSame(mlEnvironment.getStreamTableEnvironment(), streamTableEnvironment); + } + + @Test + public void testRemoveDefaultMLEnvironment() { + MLEnvironment defaultEnv = MLEnvironmentFactory.getDefault(); + MLEnvironmentFactory.remove(MLEnvironmentFactory.DEFAULT_ML_ENVIRONMENT_ID); + assertEquals("The default MLEnvironment should not have been removed", + defaultEnv, MLEnvironmentFactory.get(MLEnvironmentFactory.DEFAULT_ML_ENVIRONMENT_ID)); + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/EstimatorBaseTest.java b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/EstimatorBaseTest.java new file mode 100644 index 0000000000..17b36e25a1 --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/EstimatorBaseTest.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.ml.pipeline; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.ml.common.MLEnvironment; +import org.apache.flink.ml.common.MLEnvironmentFactory; +import org.apache.flink.ml.operator.batch.BatchOperator; +import org.apache.flink.ml.operator.stream.StreamOperator; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.api.Table; + +import org.junit.Assert; +import org.junit.Test; + +/** + * Test for {@link EstimatorBase}. + */ +public class EstimatorBaseTest extends PipelineStageTestBase { + + /** + * This fake estimator simply record which fit method is invoked. + */ + private static class FakeEstimator extends EstimatorBase { + + boolean batchFitted = false; + boolean streamFitted = false; + + @Override + public ModelBase fit(BatchOperator input) { + batchFitted = true; + return null; + } + + @Override + public ModelBase fit(StreamOperator input) { + streamFitted = true; + return null; + } + } + + @Override + protected PipelineStageBase createPipelineStage() { + return new FakeEstimator(); + } + + @Test + public void testFitBatchTable() { + Long id = MLEnvironmentFactory.getNewMLEnvironmentId(); + MLEnvironment env = MLEnvironmentFactory.get(id); + DataSet input = env.getExecutionEnvironment().fromElements(1, 2, 3); + Table table = env.getBatchTableEnvironment().fromDataSet(input); + + FakeEstimator estimator = new FakeEstimator(); + estimator.setMLEnvironmentId(id); + estimator.fit(env.getBatchTableEnvironment(), table); + + Assert.assertTrue(estimator.batchFitted); + Assert.assertFalse(estimator.streamFitted); + } + + @Test + public void testFitStreamTable() { + Long id = MLEnvironmentFactory.getNewMLEnvironmentId(); + MLEnvironment env = MLEnvironmentFactory.get(id); + DataStream input = env.getStreamExecutionEnvironment().fromElements(1, 2, 3); + Table table = env.getStreamTableEnvironment().fromDataStream(input); + + FakeEstimator estimator = new FakeEstimator(); + estimator.setMLEnvironmentId(id); + estimator.fit(env.getStreamTableEnvironment(), table); + + Assert.assertFalse(estimator.batchFitted); + Assert.assertTrue(estimator.streamFitted); + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/PipelineStageTestBase.java b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/PipelineStageTestBase.java new file mode 100644 index 0000000000..292bffcda7 --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/PipelineStageTestBase.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.ml.pipeline; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.ml.api.core.Estimator; +import org.apache.flink.ml.api.core.Transformer; +import org.apache.flink.ml.common.MLEnvironment; +import org.apache.flink.ml.common.MLEnvironmentFactory; +import org.apache.flink.table.api.Table; + +import org.junit.Test; + +/** + * The base class for testing the base implementation of pipeline stages, i.e. Estimators and Transformers. + * This class is package private because we do not expect extension outside of the package. + */ +abstract class PipelineStageTestBase { + + @Test(expected = IllegalArgumentException.class) + public void testMismatchTableEnvironment() { + Long id = MLEnvironmentFactory.getNewMLEnvironmentId(); + MLEnvironment env = MLEnvironmentFactory.get(id); + DataSet input = env.getExecutionEnvironment().fromElements(1, 2, 3); + Table t = env.getBatchTableEnvironment().fromDataSet(input); + + PipelineStageBase pipelineStageBase = createPipelineStage(); + pipelineStageBase.setMLEnvironmentId(id); + if (pipelineStageBase instanceof EstimatorBase) { + ((Estimator) pipelineStageBase).fit(MLEnvironmentFactory.getDefault().getBatchTableEnvironment(), t); + } else { + ((Transformer) pipelineStageBase).transform(MLEnvironmentFactory.getDefault().getBatchTableEnvironment(), t); + } + } + + @Test(expected = IllegalArgumentException.class) + public void testNullInputTable() { + Long id = MLEnvironmentFactory.getNewMLEnvironmentId(); + MLEnvironment env = MLEnvironmentFactory.get(id); + + PipelineStageBase pipelineStageBase = createPipelineStage(); + pipelineStageBase.setMLEnvironmentId(id); + if (pipelineStageBase instanceof Estimator) { + ((Estimator) pipelineStageBase).fit(env.getBatchTableEnvironment(), null); + } else { + ((Transformer) pipelineStageBase).transform(env.getBatchTableEnvironment(), null); + } + } + + protected abstract PipelineStageBase createPipelineStage(); +} diff --git a/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/TransformerBaseTest.java b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/TransformerBaseTest.java new file mode 100644 index 0000000000..2797685a76 --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/TransformerBaseTest.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.ml.pipeline; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.ml.common.MLEnvironment; +import org.apache.flink.ml.common.MLEnvironmentFactory; +import org.apache.flink.ml.operator.batch.BatchOperator; +import org.apache.flink.ml.operator.stream.StreamOperator; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.api.Table; + +import org.junit.Assert; +import org.junit.Test; + +/** + * Unit test for {@link TransformerBase}. + */ +public class TransformerBaseTest extends PipelineStageTestBase { + + /** + * This fake transformer simply record which transform method is invoked. + */ + private static class FakeTransFormer extends TransformerBase { + + boolean batchTransformed = false; + boolean streamTransformed = false; + + @Override + protected BatchOperator transform(BatchOperator input) { + batchTransformed = true; + return input; + } + + @Override + protected StreamOperator transform(StreamOperator input) { + streamTransformed = true; + return input; + } + } + + @Override + protected PipelineStageBase createPipelineStage() { + return new FakeTransFormer(); + } + + @Test + public void testFitBatchTable() { + Long id = MLEnvironmentFactory.getNewMLEnvironmentId(); + MLEnvironment env = MLEnvironmentFactory.get(id); + DataSet input = env.getExecutionEnvironment().fromElements(1, 2, 3); + Table table = env.getBatchTableEnvironment().fromDataSet(input); + + FakeTransFormer transFormer = new FakeTransFormer(); + transFormer.setMLEnvironmentId(id); + transFormer.transform(env.getBatchTableEnvironment(), table); + + Assert.assertTrue(transFormer.batchTransformed); + Assert.assertFalse(transFormer.streamTransformed); + } + + @Test + public void testFitStreamTable() { + Long id = MLEnvironmentFactory.getNewMLEnvironmentId(); + MLEnvironment env = MLEnvironmentFactory.get(id); + DataStream input = env.getStreamExecutionEnvironment().fromElements(1, 2, 3); + Table table = env.getStreamTableEnvironment().fromDataStream(input); + + FakeTransFormer transFormer = new FakeTransFormer(); + transFormer.setMLEnvironmentId(id); + transFormer.transform(env.getStreamTableEnvironment(), table); + + Assert.assertFalse(transFormer.batchTransformed); + Assert.assertTrue(transFormer.streamTransformed); + } +} -- Gitee From 3da95400b8ba2e7aadfc83d6de106b1dfa884831 Mon Sep 17 00:00:00 2001 From: wangxlong <18868816710@163.com> Date: Mon, 14 Oct 2019 16:23:50 +0800 Subject: [PATCH 196/268] [hotfix][doc] fix typos in RichFunction --- .../org/apache/flink/api/common/functions/RichFunction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java index 9c51b2b000..8bace82b64 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java @@ -39,7 +39,7 @@ public interface RichFunction extends Function { * composition. * *

{@code
-	 * public class MyMapper extends FilterFunction {
+	 * public class MyFilter extends RichFilterFunction {
 	 *
 	 *     private String searchString;
 	 *
-- 
Gitee


From 3346ab459ebf64f33b1d0775719a70ebf91eef95 Mon Sep 17 00:00:00 2001
From: Zhu Zhu 
Date: Tue, 8 Oct 2019 14:53:08 +0800
Subject: [PATCH 197/268] [hotfix][runtime] Pipelined partition consumers
 should not be scheduled in
 LazyFromSourcesSchedulingStrategy#onExecutionStateChange

The pipelined partition consumers should be already scheduled in
LazyFromSourcesSchedulingStrategy#onPartitionConsumable.
---
 .../scheduler/strategy/LazyFromSourcesSchedulingStrategy.java    | 1 +
 1 file changed, 1 insertion(+)

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategy.java
index 55a77c8e63..bc4ebb65c5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategy.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategy.java
@@ -102,6 +102,7 @@ public class LazyFromSourcesSchedulingStrategy implements SchedulingStrategy {
 		final Set verticesToSchedule = schedulingTopology.getVertexOrThrow(executionVertexId)
 			.getProducedResultPartitions()
 			.stream()
+			.filter(partition -> partition.getPartitionType().isBlocking())
 			.flatMap(partition -> inputConstraintChecker.markSchedulingResultPartitionFinished(partition).stream())
 			.flatMap(partition -> partition.getConsumers().stream())
 			.collect(Collectors.toSet());
-- 
Gitee


From b025b397704c0d523bf642e0af3ee53c82bf1698 Mon Sep 17 00:00:00 2001
From: Zhu Zhu 
Date: Wed, 9 Oct 2019 15:41:20 +0800
Subject: [PATCH 198/268] [FLINK-14331][runtime] Introduce
 TestSchedulingStrategy for flexible scheduler testing

---
 .../strategy/TestSchedulingStrategy.java      | 122 ++++++++++++++++++
 1 file changed, 122 insertions(+)
 create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestSchedulingStrategy.java

diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestSchedulingStrategy.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestSchedulingStrategy.java
new file mode 100644
index 0000000000..eebec4fec1
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestSchedulingStrategy.java
@@ -0,0 +1,122 @@
+/*
+ * 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.scheduler.strategy;
+
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.scheduler.DeploymentOption;
+import org.apache.flink.runtime.scheduler.ExecutionVertexDeploymentOption;
+import org.apache.flink.runtime.scheduler.SchedulerOperations;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link SchedulingStrategy} instance for tests.
+ */
+public class TestSchedulingStrategy implements SchedulingStrategy {
+
+	private final SchedulerOperations schedulerOperations;
+
+	private final SchedulingTopology schedulingTopology;
+
+	private final DeploymentOption deploymentOption = new DeploymentOption(false);
+
+	private Set receivedVerticesToRestart;
+
+	public TestSchedulingStrategy(
+			final SchedulerOperations schedulerOperations,
+			final SchedulingTopology schedulingTopology) {
+
+		this.schedulerOperations = checkNotNull(schedulerOperations);
+		this.schedulingTopology = checkNotNull(schedulingTopology);
+	}
+
+	@Override
+	public void startScheduling() {
+	}
+
+	@Override
+	public void restartTasks(final Set verticesToRestart) {
+		this.receivedVerticesToRestart = verticesToRestart;
+	}
+
+	@Override
+	public void onExecutionStateChange(final ExecutionVertexID executionVertexId, final ExecutionState executionState) {
+	}
+
+	@Override
+	public void onPartitionConsumable(final ExecutionVertexID executionVertexId, final ResultPartitionID resultPartitionId) {
+	}
+
+	public void schedule(final Set verticesToSchedule) {
+		allocateSlotsAndDeploy(verticesToSchedule);
+	}
+
+	public SchedulingTopology getSchedulingTopology() {
+		return schedulingTopology;
+	}
+
+	public Set getReceivedVerticesToRestart() {
+		return receivedVerticesToRestart;
+	}
+
+	private void allocateSlotsAndDeploy(final Set verticesToSchedule) {
+		final List executionVertexDeploymentOptions =
+			createExecutionVertexDeploymentOptions(verticesToSchedule);
+		schedulerOperations.allocateSlotsAndDeploy(executionVertexDeploymentOptions);
+	}
+
+	private List createExecutionVertexDeploymentOptions(
+			final Collection vertices) {
+
+		final List executionVertexDeploymentOptions = new ArrayList<>(vertices.size());
+		for (ExecutionVertexID executionVertexID : vertices) {
+			executionVertexDeploymentOptions.add(new ExecutionVertexDeploymentOption(executionVertexID, deploymentOption));
+		}
+		return executionVertexDeploymentOptions;
+	}
+
+	/**
+	 * The factory for creating {@link TestSchedulingStrategy}.
+	 */
+	public static class Factory implements SchedulingStrategyFactory {
+
+		private TestSchedulingStrategy lastInstance;
+
+		@Override
+		public SchedulingStrategy createInstance(
+				final SchedulerOperations schedulerOperations,
+				final SchedulingTopology schedulingTopology,
+				final JobGraph jobGraph) {
+
+			lastInstance = new TestSchedulingStrategy(schedulerOperations, schedulingTopology);
+			return lastInstance;
+		}
+
+		public TestSchedulingStrategy getLastCreatedSchedulingStrategy() {
+			return lastInstance;
+		}
+	}
+}
-- 
Gitee


From a09bfc417eddf3c2a65685b950f0f4120793eb19 Mon Sep 17 00:00:00 2001
From: Zhu Zhu 
Date: Fri, 11 Oct 2019 22:43:41 +0800
Subject: [PATCH 199/268] [FLINK-14331][runtime] Cancel the ongoing slot
 request of a vertex when canceling the vertex

---
 .../apache/flink/runtime/scheduler/DefaultScheduler.java    | 6 +-----
 1 file changed, 1 insertion(+), 5 deletions(-)

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
index 868cdab610..16b6847b56 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
@@ -212,6 +212,7 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio
 	}
 
 	private CompletableFuture cancelExecutionVertex(final ExecutionVertexID executionVertexId) {
+		executionSlotAllocator.cancel(executionVertexId);
 		return executionVertexOperations.cancel(getExecutionVertex(executionVertexId));
 	}
 
@@ -254,15 +255,10 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio
 	}
 
 	private void prepareToDeployVertices(final Set verticesToDeploy) {
-		cancelSlotAssignments(verticesToDeploy);
 		resetForNewExecutionIfInTerminalState(verticesToDeploy);
 		transitionToScheduled(verticesToDeploy);
 	}
 
-	private void cancelSlotAssignments(final Collection vertices) {
-		vertices.forEach(executionSlotAllocator::cancel);
-	}
-
 	private Collection allocateSlots(final Collection executionVertexDeploymentOptions) {
 		return executionSlotAllocator.allocateSlotsFor(executionVertexDeploymentOptions
 			.stream()
-- 
Gitee


From 03f58f4f0f072ba0799eace33112a1902989bc27 Mon Sep 17 00:00:00 2001
From: Zhu Zhu 
Date: Fri, 11 Oct 2019 22:46:48 +0800
Subject: [PATCH 200/268] [FLINK-14331][runtime] Reset vertices before asking
 the scheduling strategy to restart them

Without this change, the LazyFromSourcesSchedulingStrategy will fail to restart
tasks. This is because the LazyFromSourcesSchedulingStrategy only schedules
vertices in CREATED state.
---
 .../runtime/scheduler/DefaultScheduler.java   | 10 ++--
 .../scheduler/DefaultSchedulerTest.java       | 46 +++++++++++++++++--
 2 files changed, 45 insertions(+), 11 deletions(-)

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
index 16b6847b56..ce57b8bd93 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
@@ -199,6 +199,9 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio
 	private Runnable restartTasks(final Set executionVertexVersions) {
 		return () -> {
 			final Set verticesToRestart = executionVertexVersioner.getUnmodifiedExecutionVertices(executionVertexVersions);
+
+			resetForNewExecutionIfInTerminalState(verticesToRestart);
+
 			schedulingStrategy.restartTasks(verticesToRestart);
 		};
 	}
@@ -231,7 +234,7 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio
 		final Set verticesToDeploy = deploymentOptionsByVertex.keySet();
 		final Map requiredVersionByVertex = executionVertexVersioner.recordVertexModifications(verticesToDeploy);
 
-		prepareToDeployVertices(verticesToDeploy);
+		transitionToScheduled(verticesToDeploy);
 
 		final Collection slotExecutionVertexAssignments = allocateSlots(executionVertexDeploymentOptions);
 
@@ -254,11 +257,6 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio
 				Function.identity()));
 	}
 
-	private void prepareToDeployVertices(final Set verticesToDeploy) {
-		resetForNewExecutionIfInTerminalState(verticesToDeploy);
-		transitionToScheduled(verticesToDeploy);
-	}
-
 	private Collection allocateSlots(final Collection executionVertexDeploymentOptions) {
 		return executionSlotAllocator.allocateSlotsFor(executionVertexDeploymentOptions
 			.stream()
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
index a6bc095797..3d7e5d21fa 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
@@ -47,6 +47,10 @@ import org.apache.flink.runtime.rest.handler.legacy.backpressure.VoidBackPressur
 import org.apache.flink.runtime.scheduler.strategy.EagerSchedulingStrategy;
 import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
 import org.apache.flink.runtime.scheduler.strategy.LazyFromSourcesSchedulingStrategy;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategyFactory;
+import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology;
+import org.apache.flink.runtime.scheduler.strategy.TestSchedulingStrategy;
 import org.apache.flink.runtime.shuffle.NettyShuffleMaster;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.testtasks.NoOpInvokable;
@@ -61,6 +65,7 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -72,6 +77,7 @@ import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage;
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
 import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
@@ -279,6 +285,30 @@ public class DefaultSchedulerTest extends TestLogger {
 		assertThat(scheduler.requestJob().getState(), is(equalTo(JobStatus.RUNNING)));
 	}
 
+	@Test
+	public void vertexIsResetBeforeRestarted() throws Exception {
+		final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+		final TestSchedulingStrategy.Factory schedulingStrategyFactory = new TestSchedulingStrategy.Factory();
+		final DefaultScheduler scheduler = createScheduler(jobGraph, schedulingStrategyFactory);
+		final TestSchedulingStrategy schedulingStrategy = schedulingStrategyFactory.getLastCreatedSchedulingStrategy();
+		final SchedulingTopology topology = schedulingStrategy.getSchedulingTopology();
+
+		startScheduling(scheduler);
+
+		final SchedulingExecutionVertex onlySchedulingVertex = Iterables.getOnlyElement(topology.getVertices());
+		schedulingStrategy.schedule(Collections.singleton(onlySchedulingVertex.getId()));
+
+		final ArchivedExecutionVertex onlyExecutionVertex = Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices());
+		final ExecutionAttemptID attemptId = onlyExecutionVertex.getCurrentExecutionAttempt().getAttemptId();
+		scheduler.updateTaskExecutionState(new TaskExecutionState(jobGraph.getJobID(), attemptId, ExecutionState.FAILED));
+
+		taskRestartExecutor.triggerScheduledTasks();
+
+		assertThat(schedulingStrategy.getReceivedVerticesToRestart(), hasSize(1));
+		assertThat(onlySchedulingVertex.getState(), is(equalTo(ExecutionState.CREATED)));
+	}
+
 	private void waitForTermination(final DefaultScheduler scheduler) throws Exception {
 		scheduler.getTerminationFuture().get(TIMEOUT_MS, TimeUnit.MILLISECONDS);
 	}
@@ -316,8 +346,13 @@ public class DefaultSchedulerTest extends TestLogger {
 	}
 
 	private DefaultScheduler createSchedulerAndStartScheduling(final JobGraph jobGraph) {
+		final SchedulingStrategyFactory schedulingStrategyFactory =
+			jobGraph.getScheduleMode() == ScheduleMode.LAZY_FROM_SOURCES ?
+				new LazyFromSourcesSchedulingStrategy.Factory() :
+				new EagerSchedulingStrategy.Factory();
+
 		try {
-			final DefaultScheduler scheduler = createScheduler(jobGraph);
+			final DefaultScheduler scheduler = createScheduler(jobGraph, schedulingStrategyFactory);
 			startScheduling(scheduler);
 			return scheduler;
 		} catch (Exception e) {
@@ -325,7 +360,10 @@ public class DefaultSchedulerTest extends TestLogger {
 		}
 	}
 
-	private DefaultScheduler createScheduler(final JobGraph jobGraph) throws Exception {
+	private DefaultScheduler createScheduler(
+			final JobGraph jobGraph,
+			final SchedulingStrategyFactory schedulingStrategyFactory) throws Exception {
+
 		return new DefaultScheduler(
 			log,
 			jobGraph,
@@ -343,9 +381,7 @@ public class DefaultSchedulerTest extends TestLogger {
 			Time.seconds(300),
 			NettyShuffleMaster.INSTANCE,
 			NoOpPartitionTracker.INSTANCE,
-			jobGraph.getScheduleMode() == ScheduleMode.LAZY_FROM_SOURCES ?
-				new LazyFromSourcesSchedulingStrategy.Factory() :
-				new EagerSchedulingStrategy.Factory(),
+			schedulingStrategyFactory,
 			new RestartPipelinedRegionStrategy.Factory(),
 			testRestartBackoffTimeStrategy,
 			testExecutionVertexOperations,
-- 
Gitee


From 42102f2b3bec5c5ccee5debdafbe288a795c564c Mon Sep 17 00:00:00 2001
From: Zhu Zhu 
Date: Mon, 14 Oct 2019 16:12:14 +0800
Subject: [PATCH 201/268] [FLINK-14331][runtime] Make DefaultScheduler only
 schedule vertices in CREATED state

Also define SchedulerOperations#allocateSlotsAndDeploy to accept vertices in
CREATED state only.  This rule helps to figure out invalid or duplicated
scheduling requests.

This closes #9860.
---
 .../runtime/scheduler/DefaultScheduler.java   | 21 ++++++++++++---
 .../scheduler/SchedulerOperations.java        |  1 +
 .../scheduler/DefaultSchedulerTest.java       | 26 +++++++++++++++++++
 3 files changed, 45 insertions(+), 3 deletions(-)

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
index ce57b8bd93..638a633c63 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
@@ -230,13 +230,19 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio
 
 	@Override
 	public void allocateSlotsAndDeploy(final Collection executionVertexDeploymentOptions) {
-		final Map deploymentOptionsByVertex = groupDeploymentOptionsByVertexId(executionVertexDeploymentOptions);
+		validateDeploymentOptions(executionVertexDeploymentOptions);
+
+		final Map deploymentOptionsByVertex =
+			groupDeploymentOptionsByVertexId(executionVertexDeploymentOptions);
+
 		final Set verticesToDeploy = deploymentOptionsByVertex.keySet();
-		final Map requiredVersionByVertex = executionVertexVersioner.recordVertexModifications(verticesToDeploy);
+		final Map requiredVersionByVertex =
+			executionVertexVersioner.recordVertexModifications(verticesToDeploy);
 
 		transitionToScheduled(verticesToDeploy);
 
-		final Collection slotExecutionVertexAssignments = allocateSlots(executionVertexDeploymentOptions);
+		final Collection slotExecutionVertexAssignments =
+			allocateSlots(executionVertexDeploymentOptions);
 
 		final Collection deploymentHandles = createDeploymentHandles(
 			requiredVersionByVertex,
@@ -250,6 +256,15 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio
 		}
 	}
 
+	private void validateDeploymentOptions(final Collection deploymentOptions) {
+		deploymentOptions.stream()
+			.map(ExecutionVertexDeploymentOption::getExecutionVertexId)
+			.map(this::getExecutionVertex)
+			.forEach(v -> checkState(
+				v.getExecutionState() == ExecutionState.CREATED,
+				"expected vertex %s to be in CREATED state, was: %s", v.getID(), v.getExecutionState()));
+	}
+
 	private static Map groupDeploymentOptionsByVertexId(
 			final Collection executionVertexDeploymentOptions) {
 		return executionVertexDeploymentOptions.stream().collect(Collectors.toMap(
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerOperations.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerOperations.java
index 50d3f87323..3256a171e4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerOperations.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerOperations.java
@@ -29,6 +29,7 @@ public interface SchedulerOperations {
 
 	/**
 	 * Allocate slots and deploy the vertex when slots are returned.
+	 * Only vertices in CREATED state will be accepted. Errors will happen if scheduling Non-CREATED vertices.
 	 *
 	 * @param executionVertexDeploymentOptions The tasks to be deployed and deployment options
 	 */
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
index 3d7e5d21fa..5c0b60c92c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
@@ -82,6 +82,7 @@ import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Tests for {@link DefaultScheduler}.
@@ -309,6 +310,31 @@ public class DefaultSchedulerTest extends TestLogger {
 		assertThat(onlySchedulingVertex.getState(), is(equalTo(ExecutionState.CREATED)));
 	}
 
+	@Test
+	public void scheduleOnlyIfVertexIsCreated() throws Exception {
+		final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+		final TestSchedulingStrategy.Factory schedulingStrategyFactory = new TestSchedulingStrategy.Factory();
+		final DefaultScheduler scheduler = createScheduler(jobGraph, schedulingStrategyFactory);
+		final TestSchedulingStrategy schedulingStrategy = schedulingStrategyFactory.getLastCreatedSchedulingStrategy();
+		final SchedulingTopology topology = schedulingStrategy.getSchedulingTopology();
+
+		startScheduling(scheduler);
+
+		final ExecutionVertexID onlySchedulingVertexId = Iterables.getOnlyElement(topology.getVertices()).getId();
+
+		// Schedule the vertex to get it to a non-CREATED state
+		schedulingStrategy.schedule(Collections.singleton(onlySchedulingVertexId));
+
+		// The scheduling of a non-CREATED vertex will result in IllegalStateException
+		try {
+			schedulingStrategy.schedule(Collections.singleton(onlySchedulingVertexId));
+			fail("IllegalStateException should happen");
+		} catch (IllegalStateException e) {
+			// expected exception
+		}
+	}
+
 	private void waitForTermination(final DefaultScheduler scheduler) throws Exception {
 		scheduler.getTerminationFuture().get(TIMEOUT_MS, TimeUnit.MILLISECONDS);
 	}
-- 
Gitee


From 463fe96e43e04d38c677fe8f13228dc791ab3163 Mon Sep 17 00:00:00 2001
From: Zhu Zhu 
Date: Fri, 11 Oct 2019 22:33:15 +0800
Subject: [PATCH 202/268] [hotfix][runtime] DefaultExecutionSlotAllocator
 checks for duplicated slot allocation

This ensures that a restarted vertex is in a correct state to request a slot.
It also ensures that DefaultExecutionSlotAllocator will not drop the reference
of an existing pending slot request.
---
 .../DefaultExecutionSlotAllocator.java        | 11 +++++++++++
 .../DefaultExecutionSlotAllocatorTest.java    | 19 +++++++++++++++++++
 2 files changed, 30 insertions(+)

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java
index 79484082ed..b86ff890a7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java
@@ -51,6 +51,7 @@ import java.util.stream.Collectors;
 
 import static org.apache.flink.runtime.executiongraph.ExecutionVertex.MAX_DISTINCT_LOCATIONS_TO_CONSIDER;
 import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
 
 /**
  * Default {@link ExecutionSlotAllocator} which will use {@link SlotProvider} to allocate slots and
@@ -86,6 +87,8 @@ public class DefaultExecutionSlotAllocator implements ExecutionSlotAllocator {
 	public Collection allocateSlotsFor(
 			Collection executionVertexSchedulingRequirements) {
 
+		validateSchedulingRequirements(executionVertexSchedulingRequirements);
+
 		List slotExecutionVertexAssignments =
 				new ArrayList<>(executionVertexSchedulingRequirements.size());
 
@@ -136,6 +139,14 @@ public class DefaultExecutionSlotAllocator implements ExecutionSlotAllocator {
 		return slotExecutionVertexAssignments;
 	}
 
+	private void validateSchedulingRequirements(Collection schedulingRequirements) {
+		schedulingRequirements.stream()
+			.map(ExecutionVertexSchedulingRequirements::getExecutionVertexId)
+			.forEach(id -> checkState(
+				!pendingSlotAssignments.containsKey(id),
+				"BUG: vertex %s tries to allocate a slot when its previous slot request is still pending", id));
+	}
+
 	@Override
 	public void cancel(ExecutionVertexID executionVertexId) {
 		SlotExecutionVertexAssignment slotExecutionVertexAssignment = pendingSlotAssignments.get(executionVertexId);
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorTest.java
index 4088f337cf..1437b7cc3f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorTest.java
@@ -254,6 +254,25 @@ public class DefaultExecutionSlotAllocatorTest extends TestLogger {
 		assertThat(allPriorAllocationIds, containsInAnyOrder(expectAllocationIds.toArray()));
 	}
 
+	@Test
+	public void testDuplicatedSlotAllocationIsNotAllowed() {
+		final ExecutionVertexID executionVertexId = new ExecutionVertexID(new JobVertexID(), 0);
+
+		final DefaultExecutionSlotAllocator executionSlotAllocator = createExecutionSlotAllocator();
+		slotProvider.disableSlotAllocation();
+
+		final List schedulingRequirements =
+			createSchedulingRequirements(executionVertexId);
+		executionSlotAllocator.allocateSlotsFor(schedulingRequirements);
+
+		try {
+			executionSlotAllocator.allocateSlotsFor(schedulingRequirements);
+			fail("exception should happen");
+		} catch (IllegalStateException e) {
+			// IllegalStateException is expected
+		}
+	}
+
 	private DefaultExecutionSlotAllocator createExecutionSlotAllocator() {
 		return createExecutionSlotAllocator(new TestingInputsLocationsRetriever.Builder().build());
 	}
-- 
Gitee


From 51bd2f68b3d1dbf5e553e26730695915e13a34f4 Mon Sep 17 00:00:00 2001
From: Dian Fu 
Date: Mon, 14 Oct 2019 15:54:48 +0800
Subject: [PATCH 203/268] [FLINK-13999][cep][docs] Correct the example in the
 section of Aggregations of MATCH_RECOGNIZE

---
 docs/dev/table/streaming/match_recognize.md | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)

diff --git a/docs/dev/table/streaming/match_recognize.md b/docs/dev/table/streaming/match_recognize.md
index 4aeb110f6d..663c7d2a59 100644
--- a/docs/dev/table/streaming/match_recognize.md
+++ b/docs/dev/table/streaming/match_recognize.md
@@ -285,7 +285,7 @@ FROM Ticker
             LAST(A.rowtime) AS end_tstamp,
             AVG(A.price) AS avgPrice
         ONE ROW PER MATCH
-        AFTER MATCH SKIP TO FIRST B
+        AFTER MATCH SKIP PAST LAST ROW
         PATTERN (A+ B)
         DEFINE
             A AS AVG(A.price) < 15
@@ -307,19 +307,20 @@ symbol         rowtime         price    tax
 'ACME'  '01-Apr-11 10:00:07'   10      2
 'ACME'  '01-Apr-11 10:00:08'   15      2
 'ACME'  '01-Apr-11 10:00:09'   25      2
-'ACME'  '01-Apr-11 10:00:10'   30      1
+'ACME'  '01-Apr-11 10:00:10'   25      1
+'ACME'  '01-Apr-11 10:00:11'   30      1
 {% endhighlight %}
 
 The query will accumulate events as part of the pattern variable `A` as long as the average price
 of them does not exceed `15`. For example, such a limit exceeding happens at `01-Apr-11 10:00:04`.
-The following period exceeds the average price of `15` again at `01-Apr-11 10:00:10`. Thus the
+The following period exceeds the average price of `15` again at `01-Apr-11 10:00:11`. Thus the
 results for said query will be:
 
 {% highlight text %}
  symbol       start_tstamp       end_tstamp          avgPrice
 =========  ==================  ==================  ============
 ACME       01-APR-11 10:00:00  01-APR-11 10:00:03     14.5
-ACME       01-APR-11 10:00:04  01-APR-11 10:00:09     13.5
+ACME       01-APR-11 10:00:05  01-APR-11 10:00:10     13.5
 {% endhighlight %}
 
 Note Aggregations can be applied to expressions, but only if
-- 
Gitee


From 08964911de86d17761ae155d5cba2ba07cec9fba Mon Sep 17 00:00:00 2001
From: openinx 
Date: Mon, 14 Oct 2019 23:46:08 +0800
Subject: [PATCH 204/268] [hotfix][docs] Fix the incorrect scala checkstyle
 configure file path

This closes #9875
---
 docs/flinkDev/ide_setup.md    | 2 +-
 docs/flinkDev/ide_setup.zh.md | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/flinkDev/ide_setup.md b/docs/flinkDev/ide_setup.md
index a6108afa98..bb8e67665b 100644
--- a/docs/flinkDev/ide_setup.md
+++ b/docs/flinkDev/ide_setup.md
@@ -115,7 +115,7 @@ Nevertheless please make sure that code you add/modify in these modules still co
 
 ### Checkstyle For Scala
 
-Enable scalastyle in Intellij by selecting Settings -> Editor -> Inspections, then searching for "Scala style inspections". Also Place `"tools/maven/scalastyle_config.xml"` in the `"/.idea"` or `"/project"` directory.
+Enable scalastyle in Intellij by selecting Settings -> Editor -> Inspections, then searching for "Scala style inspections". Also Place `"tools/maven/scalastyle-config.xml"` in the `"/.idea"` or `"/project"` directory.
 
 ## Eclipse
 
diff --git a/docs/flinkDev/ide_setup.zh.md b/docs/flinkDev/ide_setup.zh.md
index 0b82b2d255..57d76170d3 100644
--- a/docs/flinkDev/ide_setup.zh.md
+++ b/docs/flinkDev/ide_setup.zh.md
@@ -115,7 +115,7 @@ Nevertheless please make sure that code you add/modify in these modules still co
 
 ### Checkstyle For Scala
 
-Enable scalastyle in Intellij by selecting Settings -> Editor -> Inspections, then searching for "Scala style inspections". Also Place `"tools/maven/scalastyle_config.xml"` in the `"/.idea"` or `"/project"` directory.
+Enable scalastyle in Intellij by selecting Settings -> Editor -> Inspections, then searching for "Scala style inspections". Also Place `"tools/maven/scalastyle-config.xml"` in the `"/.idea"` or `"/project"` directory.
 
 ## Eclipse
 
-- 
Gitee


From e2c4213a8f0664285658f25f2169487eb952909e Mon Sep 17 00:00:00 2001
From: yanghua 
Date: Wed, 18 Sep 2019 20:05:05 +0800
Subject: [PATCH 205/268] [FLINK-13025] Elasticsearch 7.x support

---
 docs/dev/connectors/elasticsearch.md          |  11 +-
 docs/dev/connectors/elasticsearch.zh.md       |   5 +
 .../ElasticsearchUpsertTableSinkBase.java     |   3 +
 .../descriptors/ElasticsearchValidator.java   |   4 +-
 .../flink-connector-elasticsearch6/pom.xml    |   8 -
 .../flink-connector-elasticsearch7/pom.xml    | 190 ++++++++++++
 .../Elasticsearch7ApiCallBridge.java          | 143 +++++++++
 .../Elasticsearch7BulkProcessorIndexer.java   |  85 ++++++
 .../elasticsearch7/ElasticsearchSink.java     | 238 +++++++++++++++
 .../elasticsearch7/RestClientFactory.java     |  40 +++
 ...eddedElasticsearchNodeEnvironmentImpl.java |  79 +++++
 .../ElasticsearchSinkITCase.java              | 100 ++++++
 .../src/test/resources/log4j-test.properties  |  24 ++
 .../pom.xml                                   | 205 +++++++++++++
 .../Elasticsearch7UpsertTableSink.java        | 285 ++++++++++++++++++
 .../Elasticsearch7UpsertTableSinkFactory.java |  76 +++++
 .../src/main/resources/META-INF/NOTICE        |  45 +++
 ....apache.flink.table.factories.TableFactory |  16 +
 ...sticsearch7UpsertTableSinkFactoryTest.java | 217 +++++++++++++
 .../src/test/resources/log4j-test.properties  |  24 ++
 flink-connectors/pom.xml                      |   2 +
 .../flink-elasticsearch7-test/pom.xml         |  92 ++++++
 .../tests/Elasticsearch7SinkExample.java      | 150 +++++++++
 flink-end-to-end-tests/pom.xml                |   1 +
 tools/travis/stage.sh                         |   2 +
 25 files changed, 2032 insertions(+), 13 deletions(-)
 create mode 100644 flink-connectors/flink-connector-elasticsearch7/pom.xml
 create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java
 create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java
 create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java
 create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java
 create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java
 create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java
 create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j-test.properties
 create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/pom.xml
 create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java
 create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java
 create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE
 create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory
 create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java
 create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties
 create mode 100644 flink-end-to-end-tests/flink-elasticsearch7-test/pom.xml
 create mode 100644 flink-end-to-end-tests/flink-elasticsearch7-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch7SinkExample.java

diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md
index 597b89b921..23cdb8e269 100644
--- a/docs/dev/connectors/elasticsearch.md
+++ b/docs/dev/connectors/elasticsearch.md
@@ -53,7 +53,12 @@ of the Elasticsearch installation:
     
         flink-connector-elasticsearch6{{ site.scala_version_suffix }}
         1.6.0
-        6 and later versions
+        6.x
+    
+    
+        flink-connector-elasticsearch7{{ site.scala_version_suffix }}
+        1.10.0
+        7 and later versions
     
   
 
@@ -123,7 +128,7 @@ input.addSink(new ElasticsearchSink<>(config, transportAddresses, new Elasticsea
     }
 }));{% endhighlight %}
 
-
+
{% highlight java %} import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.streaming.api.datastream.DataStream; @@ -226,7 +231,7 @@ input.addSink(new ElasticsearchSink(config, transportAddresses, new Elasticsearc })) {% endhighlight %}
-
+
{% highlight scala %} import org.apache.flink.api.common.functions.RuntimeContext import org.apache.flink.streaming.api.datastream.DataStream diff --git a/docs/dev/connectors/elasticsearch.zh.md b/docs/dev/connectors/elasticsearch.zh.md index 1fba302c30..e816da873e 100644 --- a/docs/dev/connectors/elasticsearch.zh.md +++ b/docs/dev/connectors/elasticsearch.zh.md @@ -55,6 +55,11 @@ of the Elasticsearch installation: 1.6.0 6 and later versions + + flink-connector-elasticsearch7{{ site.scala_version_suffix }} + 1.10.0 + 7 and later versions + diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java index c2c5181d4d..eba7ead105 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -377,6 +377,7 @@ public abstract class ElasticsearchUpsertTableSinkBase implements UpsertStreamTa /** * Creates an update request to be added to a {@link RequestIndexer}. + * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. */ UpdateRequest createUpdateRequest( String index, @@ -387,6 +388,7 @@ public abstract class ElasticsearchUpsertTableSinkBase implements UpsertStreamTa /** * Creates an index request to be added to a {@link RequestIndexer}. + * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. */ IndexRequest createIndexRequest( String index, @@ -396,6 +398,7 @@ public abstract class ElasticsearchUpsertTableSinkBase implements UpsertStreamTa /** * Creates a delete request to be added to a {@link RequestIndexer}. + * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. */ DeleteRequest createDeleteRequest( String index, diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java index 9d1a926efd..ec48557c1c 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java @@ -21,7 +21,6 @@ package org.apache.flink.table.descriptors; import org.apache.flink.annotation.Internal; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; @@ -36,6 +35,7 @@ public class ElasticsearchValidator extends ConnectorDescriptorValidator { public static final String CONNECTOR_TYPE_VALUE_ELASTICSEARCH = "elasticsearch"; public static final String CONNECTOR_VERSION_VALUE_6 = "6"; + public static final String CONNECTOR_VERSION_VALUE_7 = "7"; public static final String CONNECTOR_HOSTS = "connector.hosts"; public static final String CONNECTOR_HOSTS_HOSTNAME = "hostname"; public static final String CONNECTOR_HOSTS_PORT = "port"; @@ -79,7 +79,7 @@ public class ElasticsearchValidator extends ConnectorDescriptorValidator { properties.validateEnumValues( CONNECTOR_VERSION, false, - Collections.singletonList(CONNECTOR_VERSION_VALUE_6)); + Arrays.asList(CONNECTOR_VERSION_VALUE_6, CONNECTOR_VERSION_VALUE_7)); } private void validateHosts(DescriptorProperties properties) { diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index cc0bbf2574..5bdf889e1b 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -93,14 +93,6 @@ under the License. provided true - - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${project.version} - provided - true - diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml new file mode 100644 index 0000000000..3c68eac4fe --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -0,0 +1,190 @@ + + + + + 4.0.0 + + + flink-connectors + org.apache.flink + 1.10-SNAPSHOT + .. + + + flink-connector-elasticsearch7_${scala.binary.version} + flink-connector-elasticsearch7 + + jar + + + + 7.3.2 + + + + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + provided + + + + org.apache.flink + flink-connector-elasticsearch-base_${scala.binary.version} + ${project.version} + + + + org.elasticsearch + elasticsearch + + + + + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${elasticsearch.version} + + + + + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${project.version} + provided + true + + + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + test + test-jar + + + + org.apache.flink + flink-connector-elasticsearch-base_${scala.binary.version} + ${project.version} + + + org.elasticsearch + elasticsearch + + + test-jar + test + + + + + + org.elasticsearch.client + transport + ${elasticsearch.version} + test + + + + org.elasticsearch.plugin + transport-netty4-client + ${elasticsearch.version} + test + + + + + + org.apache.logging.log4j + log4j-core + 2.9.1 + test + + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test-jar + test + + + + + org.apache.flink + flink-json + ${project.version} + test + + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.12.2 + + + true + + + org.apache.logging.log4j:log4j-to-slf4j + + + + + + + diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java new file mode 100644 index 0000000000..c569f3b810 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java @@ -0,0 +1,143 @@ +/* + * 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.elasticsearch7; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.util.Preconditions; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.TimeValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 7 and later versions. + */ +@Internal +public class Elasticsearch7ApiCallBridge implements ElasticsearchApiCallBridge { + + private static final long serialVersionUID = -5222683870097809633L; + + private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch7ApiCallBridge.class); + + /** + * User-provided HTTP Host. + */ + private final List httpHosts; + + /** + * The factory to configure the rest client. + */ + private final RestClientFactory restClientFactory; + + Elasticsearch7ApiCallBridge(List httpHosts, RestClientFactory restClientFactory) { + Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); + this.httpHosts = httpHosts; + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } + + @Override + public RestHighLevelClient createClient(Map clientConfig) throws IOException { + RestClientBuilder builder = RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); + restClientFactory.configureRestClientBuilder(builder); + + RestHighLevelClient rhlClient = new RestHighLevelClient(builder); + + if (LOG.isInfoEnabled()) { + LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); + } + + if (!rhlClient.ping(RequestOptions.DEFAULT)) { + throw new RuntimeException("There are no reachable Elasticsearch nodes!"); + } + + if (LOG.isInfoEnabled()) { + LOG.info("Created Elasticsearch RestHighLevelClient connected to {}", httpHosts.toString()); + } + + return rhlClient; + } + + @Override + public BulkProcessor.Builder createBulkProcessorBuilder(RestHighLevelClient client, BulkProcessor.Listener listener) { + return BulkProcessor.builder((request, bulkListener) -> client.bulkAsync(request, RequestOptions.DEFAULT, bulkListener), listener); + } + + @Override + public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { + if (!bulkItemResponse.isFailed()) { + return null; + } else { + return bulkItemResponse.getFailure().getCause(); + } + } + + @Override + public void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { + + BackoffPolicy backoffPolicy; + if (flushBackoffPolicy != null) { + switch (flushBackoffPolicy.getBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + break; + case EXPONENTIAL: + default: + backoffPolicy = BackoffPolicy.exponentialBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + } + } else { + backoffPolicy = BackoffPolicy.noBackoff(); + } + + builder.setBackoffPolicy(backoffPolicy); + } + + @Override + public RequestIndexer createBulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + return new Elasticsearch7BulkProcessorIndexer( + bulkProcessor, + flushOnCheckpoint, + numPendingRequestsRef); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java new file mode 100644 index 0000000000..0b7a43741d --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java @@ -0,0 +1,85 @@ +/* + * 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.elasticsearch7; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkProcessor; +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 BulkProcessor}. + * {@link ActionRequest ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. + * + *

Note: This class is binary compatible to Elasticsearch 7. + */ +@Internal +class Elasticsearch7BulkProcessorIndexer implements RequestIndexer { + + private final BulkProcessor bulkProcessor; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; + + Elasticsearch7BulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + this.bulkProcessor = checkNotNull(bulkProcessor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(updateRequest); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java new file mode 100644 index 0000000000..5b874e2949 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java @@ -0,0 +1,238 @@ +/* + * 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.elasticsearch7; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.util.Preconditions; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RestHighLevelClient; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Elasticsearch 7.x sink that requests multiple {@link ActionRequest ActionRequests} + * against a cluster for each incoming element. + * + *

The sink internally uses a {@link RestHighLevelClient} to communicate with an Elasticsearch cluster. + * The sink will fail if no cluster can be connected to using the provided transport addresses passed to the constructor. + * + *

Internally, the sink will use a {@link BulkProcessor} to send {@link ActionRequest ActionRequests}. + * This will buffer elements before sending a request to the cluster. The behaviour of the + * {@code BulkProcessor} can be configured using these config keys: + *

    + *
  • {@code bulk.flush.max.actions}: Maximum amount of elements to buffer + *
  • {@code bulk.flush.max.size.mb}: Maximum amount of data (in megabytes) to buffer + *
  • {@code bulk.flush.interval.ms}: Interval at which to flush data regardless of the other two + * settings in milliseconds + *
+ * + *

You also have to provide an {@link ElasticsearchSinkFunction}. This is used to create multiple + * {@link ActionRequest ActionRequests} for each incoming element. See the class level documentation of + * {@link ElasticsearchSinkFunction} for an example. + * + * @param Type of the elements handled by this sink + */ +@PublicEvolving +public class ElasticsearchSink extends ElasticsearchSinkBase { + + private static final long serialVersionUID = 1L; + + private ElasticsearchSink( + Map bulkRequestsConfig, + List httpHosts, + ElasticsearchSinkFunction elasticsearchSinkFunction, + ActionRequestFailureHandler failureHandler, + RestClientFactory restClientFactory) { + + super(new Elasticsearch7ApiCallBridge(httpHosts, restClientFactory), bulkRequestsConfig, elasticsearchSinkFunction, failureHandler); + } + + /** + * A builder for creating an {@link ElasticsearchSink}. + * + * @param Type of the elements handled by the sink this builder creates. + */ + @PublicEvolving + public static class Builder { + + private final List httpHosts; + private final ElasticsearchSinkFunction elasticsearchSinkFunction; + + private Map bulkRequestsConfig = new HashMap<>(); + private ActionRequestFailureHandler failureHandler = new NoOpFailureHandler(); + private RestClientFactory restClientFactory = restClientBuilder -> {}; + + /** + * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. + * + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element. + */ + public Builder(List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { + this.httpHosts = Preconditions.checkNotNull(httpHosts); + this.elasticsearchSinkFunction = Preconditions.checkNotNull(elasticsearchSinkFunction); + } + + /** + * Sets the maximum number of actions to buffer for each bulk request. + * + * @param numMaxActions the maxinum number of actions to buffer per bulk request. + */ + public void setBulkFlushMaxActions(int numMaxActions) { + Preconditions.checkArgument( + numMaxActions > 0, + "Max number of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); + } + + /** + * Sets the maximum size of buffered actions, in mb, per bulk request. + * + * @param maxSizeMb the maximum size of buffered actions, in mb. + */ + public void setBulkFlushMaxSizeMb(int maxSizeMb) { + Preconditions.checkArgument( + maxSizeMb > 0, + "Max size of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); + } + + /** + * Sets the bulk flush interval, in milliseconds. + * + * @param intervalMillis the bulk flush interval, in milliseconds. + */ + public void setBulkFlushInterval(long intervalMillis) { + Preconditions.checkArgument( + intervalMillis >= 0, + "Interval (in milliseconds) between each flush must be larger than or equal to 0."); + + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); + } + + /** + * Sets whether or not to enable bulk flush backoff behaviour. + * + * @param enabled whether or not to enable backoffs. + */ + public void setBulkFlushBackoff(boolean enabled) { + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, String.valueOf(enabled)); + } + + /** + * Sets the type of back of to use when flushing bulk requests. + * + * @param flushBackoffType the backoff type to use. + */ + public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, + Preconditions.checkNotNull(flushBackoffType).toString()); + } + + /** + * Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk requests + */ + public void setBulkFlushBackoffRetries(int maxRetries) { + Preconditions.checkArgument( + maxRetries > 0, + "Max number of backoff attempts must be larger than 0."); + + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, String.valueOf(maxRetries)); + } + + /** + * Sets the amount of delay between each backoff attempt when flushing bulk requests, in milliseconds. + * + * @param delayMillis the amount of delay between each backoff attempt when flushing bulk requests, in milliseconds. + */ + public void setBulkFlushBackoffDelay(long delayMillis) { + Preconditions.checkArgument( + delayMillis >= 0, + "Delay (in milliseconds) between each backoff attempt must be larger than or equal to 0."); + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, String.valueOf(delayMillis)); + } + + /** + * Sets a failure handler for action requests. + * + * @param failureHandler This is used to handle failed {@link ActionRequest}. + */ + public void setFailureHandler(ActionRequestFailureHandler failureHandler) { + this.failureHandler = Preconditions.checkNotNull(failureHandler); + } + + /** + * Sets a REST client factory for custom client configuration. + * + * @param restClientFactory the factory that configures the rest client. + */ + public void setRestClientFactory(RestClientFactory restClientFactory) { + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } + + /** + * Creates the Elasticsearch sink. + * + * @return the created Elasticsearch sink. + */ + public ElasticsearchSink build() { + return new ElasticsearchSink<>(bulkRequestsConfig, httpHosts, elasticsearchSinkFunction, failureHandler, restClientFactory); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Builder builder = (Builder) o; + return Objects.equals(httpHosts, builder.httpHosts) && + Objects.equals(elasticsearchSinkFunction, builder.elasticsearchSinkFunction) && + Objects.equals(bulkRequestsConfig, builder.bulkRequestsConfig) && + Objects.equals(failureHandler, builder.failureHandler) && + Objects.equals(restClientFactory, builder.restClientFactory); + } + + @Override + public int hashCode() { + return Objects.hash( + httpHosts, + elasticsearchSinkFunction, + bulkRequestsConfig, + failureHandler, + restClientFactory); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java new file mode 100644 index 0000000000..6001f43c8b --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch7; + +import org.apache.flink.annotation.PublicEvolving; + +import org.elasticsearch.client.RestClientBuilder; + +import java.io.Serializable; + +/** + * A factory that is used to configure the {@link org.elasticsearch.client.RestHighLevelClient} internally + * used in the {@link ElasticsearchSink}. + */ +@PublicEvolving +public interface RestClientFactory extends Serializable { + + /** + * Configures the rest client builder. + * + * @param restClientBuilder the configured rest client builder. + */ + void configureRestClientBuilder(RestClientBuilder restClientBuilder); + +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java new file mode 100644 index 0000000000..6f051a383b --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java @@ -0,0 +1,79 @@ +/* + * 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.streaming.connectors.elasticsearch7.ElasticsearchSinkITCase; + +import org.elasticsearch.client.Client; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.node.InternalSettingsPreparer; +import org.elasticsearch.node.Node; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.transport.Netty4Plugin; + +import java.io.File; +import java.util.Collections; + +/** + * Implementation of {@link EmbeddedElasticsearchNodeEnvironment} for Elasticsearch 7. + * Will be dynamically loaded in {@link ElasticsearchSinkITCase} for integration tests. + */ +public class EmbeddedElasticsearchNodeEnvironmentImpl implements EmbeddedElasticsearchNodeEnvironment { + + private Node node; + + @Override + public void start(File tmpDataFolder, String clusterName) throws Exception { + if (node == null) { + Settings settings = Settings.builder() + .put("cluster.name", clusterName) + .put("http.cors.enabled", true) + .put("path.home", tmpDataFolder.getParent()) + .put("path.data", tmpDataFolder.getAbsolutePath()) + .build(); + + node = new PluginNode(settings); + node.start(); + } + } + + @Override + public void close() throws Exception { + if (node != null && !node.isClosed()) { + node.close(); + node = null; + } + } + + @Override + public Client getClient() { + if (node != null && !node.isClosed()) { + return node.client(); + } else { + return null; + } + } + + private static class PluginNode extends Node { + public PluginNode(Settings settings) { + super(InternalSettingsPreparer.prepareEnvironment(settings, Collections.emptyMap(), null, () -> "node1"), Collections.>singletonList(Netty4Plugin.class), true); + } + } + +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java new file mode 100644 index 0000000000..c8de4473a7 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.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.connectors.elasticsearch7; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; + +import org.apache.http.HttpHost; +import org.elasticsearch.client.RestHighLevelClient; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +/** + * IT cases for the {@link ElasticsearchSink}. + * + *

The Elasticsearch ITCases for 7.x CANNOT be executed in the IDE directly, since it is required that the + * Log4J-to-SLF4J adapter dependency must be excluded from the test classpath for the Elasticsearch embedded + * node used in the tests to work properly. + */ +public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { + + @Test + public void testElasticsearchSink() throws Exception { + runElasticsearchSinkTest(); + } + + @Test + public void testNullAddresses() throws Exception { + runNullAddressesTest(); + } + + @Test + public void testEmptyAddresses() throws Exception { + runEmptyAddressesTest(); + } + + @Test + public void testInvalidElasticsearchCluster() throws Exception{ + runInvalidElasticsearchClusterTest(); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSink( + int bulkFlushMaxActions, + String clusterName, + List httpHosts, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { + + ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + + return builder.build(); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForEmbeddedNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { + + return createElasticsearchSinkForNode( + bulkFlushMaxActions, clusterName, elasticsearchSinkFunction, "127.0.0.1"); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String ipAddress) throws Exception { + + ArrayList httpHosts = new ArrayList<>(); + httpHosts.add(new HttpHost(ipAddress, 9200, "http")); + + ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + + return builder.build(); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j-test.properties new file mode 100644 index 0000000000..fcd8654666 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j-test.properties @@ -0,0 +1,24 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +log4j.rootLogger=INFO, 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-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml new file mode 100644 index 0000000000..d1d8186576 --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -0,0 +1,205 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.10-SNAPSHOT + .. + + + flink-sql-connector-elasticsearch7_${scala.binary.version} + flink-sql-connector-elasticsearch7 + + jar + + + + org.apache.flink + flink-connector-elasticsearch-base_${scala.binary.version} + ${project.version} + + + + org.elasticsearch + elasticsearch + + + + + + org.apache.flink + flink-connector-elasticsearch7_${scala.binary.version} + ${project.version} + + + + + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${project.version} + provided + true + + + + org.apache.flink + flink-connector-elasticsearch-base_${scala.binary.version} + ${project.version} + + + org.elasticsearch + elasticsearch + + + test-jar + test + + + + + org.apache.flink + flink-json + ${project.version} + test + + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test-jar + test + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + + + *:* + + + + 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 + + config/** + modules.txt + plugins.txt + org/joda/** + + + + org.elasticsearch.client:elasticsearch-rest-high-level-client + + forbidden/** + + + + 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/LICENSE.txt + + + + + + + org.apache.commons + org.apache.flink.elasticsearch7.shaded.org.apache.commons + + + org.apache.http + org.apache.flink.elasticsearch7.shaded.org.apache.http + + + org.apache.lucene + org.apache.flink.elasticsearch7.shaded.org.apache.lucene + + + org.elasticsearch + org.apache.flink.elasticsearch7.shaded.org.elasticsearch + + + org.apache.logging + org.apache.flink.elasticsearch7.shaded.org.apache.logging + + + com.fasterxml.jackson + org.apache.flink.elasticsearch7.shaded.com.fasterxml.jackson + + + + + + + + + + diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java new file mode 100644 index 0000000000..b6cf75c9ed --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java @@ -0,0 +1,285 @@ +/* + * 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.sql.connectors.elasticsearch7; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; +import org.apache.flink.streaming.connectors.elasticsearch7.RestClientFactory; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_DELAY; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_ENABLED; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_RETRIES; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_TYPE; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_INTERVAL; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_MAX_ACTIONS; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_MAX_SIZE; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.DISABLE_FLUSH_ON_CHECKPOINT; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.REST_PATH_PREFIX; + +/** + * Version-specific upsert table sink for Elasticsearch 7. + */ +@Internal +public class Elasticsearch7UpsertTableSink extends ElasticsearchUpsertTableSinkBase { + + @VisibleForTesting + static final RequestFactory UPDATE_REQUEST_FACTORY = + new Elasticsearch7RequestFactory(); + + public Elasticsearch7UpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + + super( + isAppendOnly, + schema, + hosts, + index, + "", + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions, + UPDATE_REQUEST_FACTORY); + } + + @VisibleForTesting + Elasticsearch7UpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + + super( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions, + UPDATE_REQUEST_FACTORY); + } + + @Override + protected ElasticsearchUpsertTableSinkBase copy( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + RequestFactory requestFactory) { + + return new Elasticsearch7UpsertTableSink( + isAppendOnly, + schema, + hosts, + index, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + @Override + protected SinkFunction> createSinkFunction( + List hosts, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + ElasticsearchUpsertSinkFunction upsertSinkFunction) { + + final List httpHosts = hosts.stream() + .map((host) -> new HttpHost(host.hostname, host.port, host.protocol)) + .collect(Collectors.toList()); + + final ElasticsearchSink.Builder> builder = createBuilder(upsertSinkFunction, httpHosts); + + builder.setFailureHandler(failureHandler); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_ACTIONS)) + .ifPresent(v -> builder.setBulkFlushMaxActions(Integer.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_SIZE)) + .ifPresent(v -> builder.setBulkFlushMaxSizeMb(MemorySize.parse(v).getMebiBytes())); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_INTERVAL)) + .ifPresent(v -> builder.setBulkFlushInterval(Long.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_ENABLED)) + .ifPresent(v -> builder.setBulkFlushBackoff(Boolean.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_TYPE)) + .ifPresent(v -> builder.setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_RETRIES)) + .ifPresent(v -> builder.setBulkFlushBackoffRetries(Integer.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_DELAY)) + .ifPresent(v -> builder.setBulkFlushBackoffDelay(Long.valueOf(v))); + + builder.setRestClientFactory( + new DefaultRestClientFactory(sinkOptions.get(REST_PATH_PREFIX))); + + final ElasticsearchSink> sink = builder.build(); + + Optional.ofNullable(sinkOptions.get(DISABLE_FLUSH_ON_CHECKPOINT)) + .ifPresent(v -> { + if (Boolean.valueOf(v)) { + sink.disableFlushOnCheckpoint(); + } + }); + + return sink; + } + + @VisibleForTesting + ElasticsearchSink.Builder> createBuilder( + ElasticsearchUpsertSinkFunction upsertSinkFunction, + List httpHosts) { + return new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction); + } + + // -------------------------------------------------------------------------------------------- + // Helper classes + // -------------------------------------------------------------------------------------------- + + /** + * Serializable {@link RestClientFactory} used by the sink. + */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private String pathPrefix; + + public DefaultRestClientFactory(@Nullable String pathPrefix) { + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the sink. + */ + private static class Elasticsearch7RequestFactory implements RequestFactory { + + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, + String docType, + XContentType contentType, + byte[] document) { + return new IndexRequest(index) + .source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, key); + } + } +} diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java new file mode 100644 index 0000000000..ca2f544991 --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.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.sql.connectors.elasticsearch7; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkFactoryBase; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.sinks.UpsertStreamTableSink; +import org.apache.flink.types.Row; + +import org.elasticsearch.common.xcontent.XContentType; + +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_VERSION_VALUE_7; + +/** + * Table factory for creating an {@link UpsertStreamTableSink} for Elasticsearch 7. + */ +@Internal +public class Elasticsearch7UpsertTableSinkFactory extends ElasticsearchUpsertTableSinkFactoryBase { + + @Override + protected String elasticsearchVersion() { + return CONNECTOR_VERSION_VALUE_7; + } + + @Override + protected ElasticsearchUpsertTableSinkBase createElasticsearchUpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + + return new Elasticsearch7UpsertTableSink( + isAppendOnly, + schema, + hosts, + index, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } +} diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000..2c7e1c039b --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -0,0 +1,45 @@ +flink-sql-connector-elasticsearch7 +Copyright 2014-2019 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.fasterxml.jackson.core:jackson-core:2.8.10 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.8.10 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.10 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.10 +- commons-codec:commons-codec:1.10 +- commons-logging:commons-logging:1.1.3 +- org.apache.httpcomponents:httpasyncclient: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.9.1 +- org.apache.logging.log4j:log4j-to-slf4j:2.9.1 +- org.apache.lucene:lucene-analyzers-common:7.3.1 +- org.apache.lucene:lucene-backward-codecs:7.3.1 +- org.apache.lucene:lucene-core:7.3.1 +- org.apache.lucene:lucene-grouping:7.3.1 +- org.apache.lucene:lucene-highlighter:7.3.1 +- org.apache.lucene:lucene-join:7.3.1 +- org.apache.lucene:lucene-memory:7.3.1 +- org.apache.lucene:lucene-misc:7.3.1 +- org.apache.lucene:lucene-queries:7.3.1 +- org.apache.lucene:lucene-queryparser:7.3.1 +- org.apache.lucene:lucene-sandbox:7.3.1 +- org.apache.lucene:lucene-spatial:7.3.1 +- org.apache.lucene:lucene-spatial-extras:7.3.1 +- org.apache.lucene:lucene-spatial3d:7.3.1 +- org.apache.lucene:lucene-suggest:7.3.1 +- org.elasticsearch:elasticsearch:7.3.2 +- org.elasticsearch:elasticsearch-cli:7.3.2 +- org.elasticsearch:elasticsearch-core:7.3.2 +- org.elasticsearch:elasticsearch-secure-sm:7.3.2 +- org.elasticsearch:elasticsearch-x-content:7.3.2 +- org.elasticsearch.client:elasticsearch-rest-client:7.3.2 +- org.elasticsearch.client:elasticsearch-rest-high-level-client:7.3.2 +- org.elasticsearch.plugin:aggs-matrix-stats-client:7.3.2 +- org.elasticsearch.plugin:parent-join-client:7.3.2 +- org.elasticsearch.plugin:rank-eval-client:7.3.2 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory new file mode 100644 index 0000000000..fe10b08d2b --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.streaming.sql.connectors.elasticsearch7.Elasticsearch7UpsertTableSinkFactory diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java b/flink-connectors/flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java new file mode 100644 index 0000000000..a047b02f9b --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.sql.connectors.elasticsearch7; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.ElasticsearchUpsertSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkFactoryTestBase; +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; + +import org.apache.http.HttpHost; +import org.elasticsearch.common.xcontent.XContentType; +import org.junit.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_VERSION_VALUE_7; +import static org.junit.Assert.assertEquals; + +/** + * Test for {@link Elasticsearch7UpsertTableSink} created by {@link Elasticsearch7UpsertTableSinkFactory}. + */ +public class Elasticsearch7UpsertTableSinkFactoryTest extends ElasticsearchUpsertTableSinkFactoryTestBase { + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + + final TestElasticsearch7UpsertTableSink testSink = new TestElasticsearch7UpsertTableSink( + false, + schema, + Collections.singletonList(new Host(ElasticsearchUpsertTableSinkFactoryTestBase.HOSTNAME, ElasticsearchUpsertTableSinkFactoryTestBase.PORT, ElasticsearchUpsertTableSinkFactoryTestBase.SCHEMA)), + ElasticsearchUpsertTableSinkFactoryTestBase.INDEX, + ElasticsearchUpsertTableSinkFactoryTestBase.DOC_TYPE, + ElasticsearchUpsertTableSinkFactoryTestBase.KEY_DELIMITER, + ElasticsearchUpsertTableSinkFactoryTestBase.KEY_NULL_LITERAL, + new JsonRowSerializationSchema(schema.toRowType()), + XContentType.JSON, + new DummyFailureHandler(), + createTestSinkOptions()); + + final DataStreamMock dataStreamMock = new DataStreamMock( + new StreamExecutionEnvironmentMock(), + Types.TUPLE(Types.BOOLEAN, schema.toRowType())); + + testSink.emitDataStream(dataStreamMock); + + final ElasticsearchSink.Builder> expectedBuilder = new ElasticsearchSink.Builder<>( + Collections.singletonList(new HttpHost(ElasticsearchUpsertTableSinkFactoryTestBase.HOSTNAME, ElasticsearchUpsertTableSinkFactoryTestBase.PORT, ElasticsearchUpsertTableSinkFactoryTestBase.SCHEMA)), + new ElasticsearchUpsertSinkFunction( + ElasticsearchUpsertTableSinkFactoryTestBase.INDEX, + ElasticsearchUpsertTableSinkFactoryTestBase.DOC_TYPE, + ElasticsearchUpsertTableSinkFactoryTestBase.KEY_DELIMITER, + ElasticsearchUpsertTableSinkFactoryTestBase.KEY_NULL_LITERAL, + new JsonRowSerializationSchema(schema.toRowType()), + XContentType.JSON, + Elasticsearch7UpsertTableSink.UPDATE_REQUEST_FACTORY, + new int[0])); + expectedBuilder.setFailureHandler(new DummyFailureHandler()); + expectedBuilder.setBulkFlushBackoff(true); + expectedBuilder.setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + expectedBuilder.setBulkFlushBackoffDelay(123); + expectedBuilder.setBulkFlushBackoffRetries(3); + expectedBuilder.setBulkFlushInterval(100); + expectedBuilder.setBulkFlushMaxActions(1000); + expectedBuilder.setBulkFlushMaxSizeMb(1); + expectedBuilder.setRestClientFactory(new Elasticsearch7UpsertTableSink.DefaultRestClientFactory("/myapp")); + + assertEquals(expectedBuilder, testSink.builder); + } + + @Override + protected String getElasticsearchVersion() { + return CONNECTOR_VERSION_VALUE_7; + } + + @Override + protected ElasticsearchUpsertTableSinkBase getExpectedTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + return new Elasticsearch7UpsertTableSink( + isAppendOnly, + schema, + hosts, + index, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + // -------------------------------------------------------------------------------------------- + // Helper classes + // -------------------------------------------------------------------------------------------- + + private static class TestElasticsearch7UpsertTableSink extends Elasticsearch7UpsertTableSink { + + public ElasticsearchSink.Builder> builder; + + public TestElasticsearch7UpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + + super( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + @Override + protected ElasticsearchSink.Builder> createBuilder( + ElasticsearchUpsertSinkFunction upsertSinkFunction, + List httpHosts) { + builder = super.createBuilder(upsertSinkFunction, httpHosts); + return builder; + } + } + + private static class StreamExecutionEnvironmentMock extends StreamExecutionEnvironment { + + @Override + public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { + throw new UnsupportedOperationException(); + } + } + + private static class DataStreamMock extends DataStream> { + + public SinkFunction sinkFunction; + + public DataStreamMock(StreamExecutionEnvironment environment, TypeInformation> outType) { + super(environment, new TransformationMock("name", outType, 1)); + } + + @Override + public DataStreamSink> addSink(SinkFunction> sinkFunction) { + this.sinkFunction = sinkFunction; + return super.addSink(sinkFunction); + } + } + + private static class TransformationMock extends Transformation> { + + public TransformationMock(String name, TypeInformation> outputType, int parallelism) { + super(name, outputType, parallelism); + } + + @Override + public Collection> getTransitivePredecessors() { + return null; + } + } +} diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties b/flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties new file mode 100644 index 0000000000..fcd8654666 --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties @@ -0,0 +1,24 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +log4j.rootLogger=INFO, 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-connectors/pom.xml b/flink-connectors/pom.xml index 41a5eaa912..2c2b053b5a 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -49,6 +49,7 @@ under the License. flink-connector-elasticsearch2 flink-connector-elasticsearch5 flink-connector-elasticsearch6 + flink-connector-elasticsearch7 flink-connector-hive flink-connector-rabbitmq flink-connector-twitter @@ -58,6 +59,7 @@ under the License. flink-connector-kafka flink-connector-gcp-pubsub flink-connector-kinesis + flink-sql-connector-elasticsearch7 + + + 4.0.0 + + + org.apache.flink + flink-end-to-end-tests + 1.10-SNAPSHOT + .. + + + flink-elasticsearch7-test + flink-elasticsearch7-test + jar + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + provided + + + org.apache.flink + flink-connector-elasticsearch7_${scala.binary.version} + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + Elasticsearch7SinkExample + package + + shade + + + Elasticsearch7SinkExample + + + com.google.code.findbugs:jsr305 + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + org.apache.flink.streaming.tests.Elasticsearch7SinkExample + + + + + + + + + + diff --git a/flink-end-to-end-tests/flink-elasticsearch7-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch7SinkExample.java b/flink-end-to-end-tests/flink-elasticsearch7-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch7SinkExample.java new file mode 100644 index 0000000000..e7036b5d39 --- /dev/null +++ b/flink-end-to-end-tests/flink-elasticsearch7-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch7SinkExample.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.streaming.tests; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.tuple.Tuple2; +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.elasticsearch7.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; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * End to end test for Elasticsearch6Sink. + */ +public class Elasticsearch7SinkExample { + + public static void main(String[] args) throws Exception { + + final ParameterTool parameterTool = ParameterTool.fromArgs(args); + + if (parameterTool.getNumberOfParameters() < 2) { + System.out.println("Missing parameters!\n" + + "Usage: --numRecords --index "); + return; + } + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(5000); + + DataStream> source = env.generateSequence(0, parameterTool.getInt("numRecords") - 1) + .flatMap(new FlatMapFunction>() { + @Override + public void flatMap(Long value, Collector> out) { + final String key = String.valueOf(value); + final String message = "message #" + value; + out.collect(Tuple2.of(key, message + "update #1")); + out.collect(Tuple2.of(key, message + "update #2")); + } + }); + + List httpHosts = new ArrayList<>(); + httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); + + ElasticsearchSink.Builder> esSinkBuilder = new ElasticsearchSink.Builder<>( + httpHosts, + (Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) -> { + indexer.add(createIndexRequest(element.f1, parameterTool)); + indexer.add(createUpdateRequest(element, parameterTool)); + }); + + esSinkBuilder.setFailureHandler( + new CustomFailureHandler(parameterTool.getRequired("index"))); + + // this instructs the sink to emit after every element, otherwise they would be buffered + esSinkBuilder.setBulkFlushMaxActions(1); + + source.addSink(esSinkBuilder.build()); + + env.execute("Elasticsearch 7.x end to end sink test example"); + } + + private static class CustomFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = 942269087742453482L; + + private final String index; + + CustomFailureHandler(String index) { + this.index = index; + } + + @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) + .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"); + } + + return Requests.indexRequest() + .index(index) + .id(element) + .source(json); + } + + private static UpdateRequest createUpdateRequest(Tuple2 element, ParameterTool parameterTool) { + Map json = new HashMap<>(); + json.put("data", element.f1); + + return new UpdateRequest( + parameterTool.getRequired("index"), + parameterTool.getRequired("type"), + element.f0) + .doc(json) + .upsert(json); + } +} diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index b7ac551908..bdf04b3139 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -71,6 +71,7 @@ under the License. flink-plugins-test flink-tpch-test flink-streaming-kinesis-test + flink-elasticsearch7-test diff --git a/tools/travis/stage.sh b/tools/travis/stage.sh index 6676b46e8e..b0f4456d32 100644 --- a/tools/travis/stage.sh +++ b/tools/travis/stage.sh @@ -89,7 +89,9 @@ flink-connectors/flink-connector-cassandra,\ flink-connectors/flink-connector-elasticsearch2,\ flink-connectors/flink-connector-elasticsearch5,\ flink-connectors/flink-connector-elasticsearch6,\ +flink-connectors/flink-connector-elasticsearch7,\ flink-connectors/flink-sql-connector-elasticsearch6,\ +flink-connectors/flink-sql-connector-elasticsearch7,\ flink-connectors/flink-connector-elasticsearch-base,\ flink-connectors/flink-connector-filesystem,\ flink-connectors/flink-connector-kafka-0.9,\ -- Gitee From aa62fc289106b891c38c4b72ba3f252e63e217c4 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 14 Oct 2019 13:05:45 +0200 Subject: [PATCH 206/268] [FLINK-13025] In ES7 SQL connector, update shading rules and NOTICE --- .../pom.xml | 3 ++ .../src/main/resources/META-INF/NOTICE | 45 +++++++++---------- 2 files changed, 25 insertions(+), 23 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index d1d8186576..63aa212dcc 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -122,6 +122,9 @@ under the License. joda-time:joda-time net.sf.jopt-simple:jopt-simple org.elasticsearch:jna + org.elasticsearch:elasticsearch-geo + org.elasticsearch.plugin:lang-mustache-client + com.github.spullara.mustache.java:compiler org.hdrhistogram:HdrHistogram org.yaml:snakeyaml diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 2c7e1c039b..80866c7c2e 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -6,33 +6,32 @@ 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.fasterxml.jackson.core:jackson-core:2.8.10 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.8.10 -- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.10 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.10 +- com.fasterxml.jackson.core:jackson-core:2.8.11 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.8.11 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.11 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.11 - commons-codec:commons-codec:1.10 - commons-logging:commons-logging:1.1.3 -- org.apache.httpcomponents:httpasyncclient:4.1.2 +- org.apache.httpcomponents:httpasyncclient:4.1.4 - 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.9.1 -- org.apache.logging.log4j:log4j-to-slf4j:2.9.1 -- org.apache.lucene:lucene-analyzers-common:7.3.1 -- org.apache.lucene:lucene-backward-codecs:7.3.1 -- org.apache.lucene:lucene-core:7.3.1 -- org.apache.lucene:lucene-grouping:7.3.1 -- org.apache.lucene:lucene-highlighter:7.3.1 -- org.apache.lucene:lucene-join:7.3.1 -- org.apache.lucene:lucene-memory:7.3.1 -- org.apache.lucene:lucene-misc:7.3.1 -- org.apache.lucene:lucene-queries:7.3.1 -- org.apache.lucene:lucene-queryparser:7.3.1 -- org.apache.lucene:lucene-sandbox:7.3.1 -- org.apache.lucene:lucene-spatial:7.3.1 -- org.apache.lucene:lucene-spatial-extras:7.3.1 -- org.apache.lucene:lucene-spatial3d:7.3.1 -- org.apache.lucene:lucene-suggest:7.3.1 +- org.apache.httpcomponents:httpcore-nio:4.4.11 +- org.apache.logging.log4j:log4j-api:2.11.1 +- org.apache.lucene:lucene-analyzers-common:8.1.0 +- org.apache.lucene:lucene-backward-codecs:8.1.0 +- org.apache.lucene:lucene-core:8.1.0 +- org.apache.lucene:lucene-grouping:8.1.0 +- org.apache.lucene:lucene-highlighter:8.1.0 +- org.apache.lucene:lucene-join:8.1.0 +- org.apache.lucene:lucene-memory:8.1.0 +- org.apache.lucene:lucene-misc:8.1.0 +- org.apache.lucene:lucene-queries:8.1.0 +- org.apache.lucene:lucene-queryparser:8.1.0 +- org.apache.lucene:lucene-sandbox:8.1.0 +- org.apache.lucene:lucene-spatial:8.1.0 +- org.apache.lucene:lucene-spatial-extras:8.1.0 +- org.apache.lucene:lucene-spatial3d:8.1.0 +- org.apache.lucene:lucene-suggest:8.1.0 - org.elasticsearch:elasticsearch:7.3.2 - org.elasticsearch:elasticsearch-cli:7.3.2 - org.elasticsearch:elasticsearch-core:7.3.2 -- Gitee From 255f047f31bc5a89e2f2224bb6fd61ae4a4e44d8 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 23 Sep 2019 11:15:03 +0800 Subject: [PATCH 207/268] [hotfix] Rename `TaskManagerOptions#MANAGED_MEMORY_SIZE` and `TaskManagerOptions#MANAGED_MEMORY_FRACTION` with prefix `LEGACY_`. This is to avoid naming conflict with the new config options. --- .../kafka/KafkaShortRetentionTestBase.java | 2 +- .../connectors/kafka/KafkaTestBase.java | 2 +- .../manualtests/ManualExactlyOnceTest.java | 2 +- ...alExactlyOnceWithStreamReshardingTest.java | 2 +- .../flink/configuration/ConfigConstants.java | 6 ++--- .../configuration/TaskManagerOptions.java | 12 +++++----- ...anagerHeapSizeCalculationJavaBashTest.java | 22 ++++++++--------- .../HAQueryableStateFsBackendITCase.java | 2 +- .../HAQueryableStateRocksDBBackendITCase.java | 2 +- .../NonHAQueryableStateFsBackendITCase.java | 2 +- ...nHAQueryableStateRocksDBBackendITCase.java | 2 +- .../runtime/webmonitor/WebFrontendITCase.java | 6 ++--- .../ActiveResourceManagerFactory.java | 2 +- .../resourcemanager/ResourceManager.java | 2 +- .../taskexecutor/TaskManagerServices.java | 10 ++++---- .../TaskManagerServicesConfiguration.java | 4 ++-- .../NettyShuffleEnvironmentConfiguration.java | 4 ++-- .../util/ConfigurationParserUtils.java | 18 +++++++------- .../ActiveResourceManagerFactoryTest.java | 2 +- .../resourcemanager/ResourceManagerTest.java | 4 ++-- ...tyShuffleEnvironmentConfigurationTest.java | 6 ++--- .../NetworkBufferCalculationTest.java | 24 +++++++++---------- .../TaskManagerRunnerStartupTest.java | 4 ++-- .../testutils/MiniClusterResource.java | 4 ++-- .../environment/LocalStreamEnvironment.java | 2 +- .../gateway/local/LocalExecutorITCase.java | 2 +- .../runtime/utils/BatchAbstractTestBase.java | 2 +- .../accumulators/AccumulatorErrorITCase.java | 2 +- ...EventTimeAllWindowCheckpointingITCase.java | 2 +- .../EventTimeWindowCheckpointingITCase.java | 2 +- .../KeyedStateCheckpointingITCase.java | 2 +- .../test/checkpointing/SavepointITCase.java | 2 +- .../WindowCheckpointingITCase.java | 2 +- .../test/classloading/ClassLoaderITCase.java | 2 +- .../failing/JobSubmissionFailsITCase.java | 2 +- .../StreamingScalabilityAndLatency.java | 2 +- .../test/misc/CustomSerializationITCase.java | 2 +- ...ccessAfterNetworkBuffersFailureITCase.java | 2 +- ...TaskManagerProcessFailureRecoveryTest.java | 2 +- ...ManagerHAProcessFailureRecoveryITCase.java | 2 +- .../ProcessFailureCancelingITCase.java | 2 +- .../test/runtime/IPv6HostnamesITCase.java | 2 +- .../streaming/runtime/TimestampITCase.java | 2 +- .../flink/yarn/YarnConfigurationITCase.java | 2 +- 44 files changed, 94 insertions(+), 92 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java index e6fd77af64..ea940e0fc6 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java @@ -82,7 +82,7 @@ public class KafkaShortRetentionTestBase implements Serializable { private static Configuration getConfiguration() { Configuration flinkConfig = new Configuration(); - flinkConfig.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "16m"); + flinkConfig.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "16m"); return flinkConfig; } 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 5639c8270b..d46e21c2b0 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 @@ -117,7 +117,7 @@ public abstract class KafkaTestBase extends TestLogger { protected static Configuration getFlinkConfiguration() { Configuration flinkConfig = new Configuration(); - flinkConfig.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "16m"); + flinkConfig.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "16m"); flinkConfig.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "my_reporter." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, JMXReporter.class.getName()); return flinkConfig; } diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java index 033ead6c4c..890ac30535 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java @@ -78,7 +78,7 @@ public class ManualExactlyOnceTest { } final Configuration flinkConfig = new Configuration(); - flinkConfig.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "16m"); + flinkConfig.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "16m"); flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s"); MiniClusterResource flink = new MiniClusterResource(new MiniClusterResourceConfiguration.Builder() diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java index 26a422f7f2..8f6880a517 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java @@ -90,7 +90,7 @@ public class ManualExactlyOnceWithStreamReshardingTest { } final Configuration flinkConfig = new Configuration(); - flinkConfig.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "16m"); + flinkConfig.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "16m"); flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s"); MiniClusterResource flink = new MiniClusterResource(new MiniClusterResourceConfiguration.Builder() diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index 9a4a9e0778..fa68073d27 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -252,7 +252,7 @@ public final class ConfigConstants { * memory manager (in megabytes). If not set, a relative fraction will be allocated, as defined * by {@link #TASK_MANAGER_MEMORY_FRACTION_KEY}. * - * @deprecated Use {@link TaskManagerOptions#MANAGED_MEMORY_SIZE} instead + * @deprecated Use {@link TaskManagerOptions#LEGACY_MANAGED_MEMORY_SIZE} instead */ @Deprecated public static final String TASK_MANAGER_MEMORY_SIZE_KEY = "taskmanager.memory.size"; @@ -260,7 +260,7 @@ public final class ConfigConstants { /** * The config parameter defining the fraction of free memory allocated by the memory manager. * - * @deprecated Use {@link TaskManagerOptions#MANAGED_MEMORY_FRACTION} instead + * @deprecated Use {@link TaskManagerOptions#LEGACY_MANAGED_MEMORY_FRACTION} instead */ @Deprecated public static final String TASK_MANAGER_MEMORY_FRACTION_KEY = "taskmanager.memory.fraction"; @@ -1427,7 +1427,7 @@ public final class ConfigConstants { /** * Config key has been deprecated. Therefore, no default value required. * - * @deprecated {@link TaskManagerOptions#MANAGED_MEMORY_FRACTION} provides the default value now + * @deprecated {@link TaskManagerOptions#LEGACY_MANAGED_MEMORY_FRACTION} provides the default value now */ @Deprecated public static final float DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION = 0.7f; 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 d85a21d794..acab022b43 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 @@ -189,9 +189,9 @@ public class TaskManagerOptions { /** * Amount of memory to be allocated by the task manager's memory manager. If not - * set, a relative fraction will be allocated, as defined by {@link #MANAGED_MEMORY_FRACTION}. + * set, a relative fraction will be allocated, as defined by {@link #LEGACY_MANAGED_MEMORY_FRACTION}. */ - public static final ConfigOption MANAGED_MEMORY_SIZE = + public static final ConfigOption LEGACY_MANAGED_MEMORY_SIZE = key("taskmanager.memory.size") .defaultValue("0") .withDescription("The amount of memory (in megabytes) that the task manager reserves on-heap or off-heap" + @@ -200,10 +200,10 @@ public class TaskManagerOptions { " 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 + * Fraction of free memory allocated by the memory manager if {@link #LEGACY_MANAGED_MEMORY_SIZE} is * not set. */ - public static final ConfigOption MANAGED_MEMORY_FRACTION = + public static final ConfigOption LEGACY_MANAGED_MEMORY_FRACTION = key("taskmanager.memory.fraction") .defaultValue(0.7f) .withDescription(new Description.DescriptionBuilder() @@ -212,8 +212,8 @@ public class TaskManagerOptions { " 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")) + " created by user-defined functions. This parameter is only evaluated, if " + + LEGACY_MANAGED_MEMORY_SIZE.key() + " is not set.", code("0.8")) .build()); /** diff --git a/flink-dist/src/test/java/org/apache/flink/dist/TaskManagerHeapSizeCalculationJavaBashTest.java b/flink-dist/src/test/java/org/apache/flink/dist/TaskManagerHeapSizeCalculationJavaBashTest.java index b965355fb5..0b5f5be2ae 100755 --- a/flink-dist/src/test/java/org/apache/flink/dist/TaskManagerHeapSizeCalculationJavaBashTest.java +++ b/flink-dist/src/test/java/org/apache/flink/dist/TaskManagerHeapSizeCalculationJavaBashTest.java @@ -80,8 +80,8 @@ public class TaskManagerHeapSizeCalculationJavaBashTest extends TestLogger { final long networkBufMin = 64L << 20; // 64MB final long networkBufMax = 1L << 30; // 1GB - int managedMemSize = Integer.valueOf(TaskManagerOptions.MANAGED_MEMORY_SIZE.defaultValue()); - float managedMemFrac = TaskManagerOptions.MANAGED_MEMORY_FRACTION.defaultValue(); + int managedMemSize = Integer.valueOf(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.defaultValue()); + float managedMemFrac = TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION.defaultValue(); // manual tests from org.apache.flink.runtime.taskexecutor.TaskManagerServices.calculateHeapSizeMB() @@ -119,8 +119,8 @@ public class TaskManagerHeapSizeCalculationJavaBashTest extends TestLogger { final long networkBufMin = 64L << 20; // 64MB final long networkBufMax = 1L << 30; // 1GB - int managedMemSize = Integer.valueOf(TaskManagerOptions.MANAGED_MEMORY_SIZE.defaultValue()); - float managedMemFrac = TaskManagerOptions.MANAGED_MEMORY_FRACTION.defaultValue(); + int managedMemSize = Integer.valueOf(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.defaultValue()); + float managedMemFrac = TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION.defaultValue(); // manual tests from org.apache.flink.runtime.taskexecutor.TaskManagerServices.calculateHeapSizeMB() @@ -184,11 +184,11 @@ public class TaskManagerHeapSizeCalculationJavaBashTest extends TestLogger { config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(netBufMemMax)); if (managedMemSizeMB == 0) { - config.removeConfig(TaskManagerOptions.MANAGED_MEMORY_SIZE); + config.removeConfig(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE); } else { - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, managedMemSizeMB + "m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, managedMemSizeMB + "m"); } - config.setFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION, managedMemFrac); + config.setFloat(TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION, managedMemFrac); return config; } @@ -214,8 +214,8 @@ public class TaskManagerHeapSizeCalculationJavaBashTest extends TestLogger { int javaMemMB = Math.max((int) (max >> 20), ran.nextInt(Integer.MAX_VALUE)) + 1; boolean useOffHeap = ran.nextBoolean(); - int managedMemSize = Integer.valueOf(TaskManagerOptions.MANAGED_MEMORY_SIZE.defaultValue()); - float managedMemFrac = TaskManagerOptions.MANAGED_MEMORY_FRACTION.defaultValue(); + int managedMemSize = Integer.valueOf(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.defaultValue()); + float managedMemFrac = TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION.defaultValue(); if (ran.nextBoolean()) { // use fixed-size managed memory @@ -295,8 +295,8 @@ public class TaskManagerHeapSizeCalculationJavaBashTest extends TestLogger { String.valueOf(config.getFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION)), config.getString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN), config.getString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX), - config.getString(TaskManagerOptions.MANAGED_MEMORY_SIZE), - String.valueOf(config.getFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION))}; + config.getString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE), + String.valueOf(config.getFloat(TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION))}; String scriptOutput = executeScript(command); // we need a tolerance of at least one, to compensate for MB/byte conversion rounding errors diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java index 43fdf13a4b..de76d2bd05 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java @@ -96,7 +96,7 @@ public class HAQueryableStateFsBackendITCase extends AbstractQueryableStateTestB Configuration config = new Configuration(); config.setBoolean(QueryableStateOptions.ENABLE_QUERYABLE_STATE_PROXY_SERVER, true); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "4m"); config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, NUM_JMS); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java index 8a62243697..e170f0928b 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java @@ -96,7 +96,7 @@ public class HAQueryableStateRocksDBBackendITCase extends AbstractQueryableState Configuration config = new Configuration(); config.setBoolean(QueryableStateOptions.ENABLE_QUERYABLE_STATE_PROXY_SERVER, true); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "4m"); config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, NUM_JMS); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); 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 183ad9e59b..89797d7418 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 @@ -80,7 +80,7 @@ public class NonHAQueryableStateFsBackendITCase extends AbstractQueryableStateTe private static Configuration getConfig() { Configuration config = new Configuration(); config.setBoolean(QueryableStateOptions.ENABLE_QUERYABLE_STATE_PROXY_SERVER, true); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "4m"); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 1); diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java index 34713f38f0..3880051b2e 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java @@ -79,7 +79,7 @@ public class NonHAQueryableStateRocksDBBackendITCase extends AbstractQueryableSt private static Configuration getConfig() { Configuration config = new Configuration(); config.setBoolean(QueryableStateOptions.ENABLE_QUERYABLE_STATE_PROXY_SERVER, true); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "4m"); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 1); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java index ffee6f1ddd..21d4a85923 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java @@ -102,7 +102,7 @@ public class WebFrontendITCase extends TestLogger { } // !!DO NOT REMOVE!! next line is required for tests - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "12m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "12m"); return config; } @@ -224,8 +224,8 @@ public class WebFrontendITCase extends TestLogger { String config = TestBaseUtils.getFromHTTP("http://localhost:" + getRestPort() + "/jobmanager/config"); Map conf = WebMonitorUtils.fromKeyValueJsonArray(config); - String expected = CLUSTER_CONFIGURATION.getString(TaskManagerOptions.MANAGED_MEMORY_SIZE); - String actual = conf.get(TaskManagerOptions.MANAGED_MEMORY_SIZE.key()); + String expected = CLUSTER_CONFIGURATION.getString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE); + String actual = conf.get(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.key()); assertEquals(expected, actual); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactory.java index 483f77c25e..d292e5a6af 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactory.java @@ -75,7 +75,7 @@ public abstract class ActiveResourceManagerFactory public static Collection createWorkerSlotProfiles(Configuration config) { final int numSlots = config.getInteger(TaskManagerOptions.NUM_TASK_SLOTS); - final long managedMemoryBytes = MemorySize.parse(config.getString(TaskManagerOptions.MANAGED_MEMORY_SIZE)).getBytes(); + final long managedMemoryBytes = MemorySize.parse(config.getString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE)).getBytes(); final ResourceProfile resourceProfile = TaskManagerServices.computeSlotResourceProfile(numSlots, managedMemoryBytes); return Collections.nCopies(numSlots, resourceProfile); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java index 3db847dd34..752deabed2 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java @@ -433,7 +433,7 @@ public class TaskManagerServices { final long managedMemorySize = getManagedMemoryFromHeapAndManaged(config, heapAndManagedMemory); ConfigurationParserUtils.checkConfigParameter(managedMemorySize < heapAndManagedMemory, managedMemorySize, - TaskManagerOptions.MANAGED_MEMORY_SIZE.key(), + TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.key(), "Managed memory size too large for " + (networkReservedMemory >> 20) + " MB network buffer memory and a total of " + totalJavaMemorySizeMB + " MB JVM memory"); @@ -460,20 +460,20 @@ public class TaskManagerServices { * All values are in bytes. */ public static long getManagedMemoryFromHeapAndManaged(Configuration config, long heapAndManagedMemory) { - if (config.contains(TaskManagerOptions.MANAGED_MEMORY_SIZE)) { + if (config.contains(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE)) { // take the configured absolute value - final String sizeValue = config.getString(TaskManagerOptions.MANAGED_MEMORY_SIZE); + final String sizeValue = config.getString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE); try { return MemorySize.parse(sizeValue, MEGA_BYTES).getBytes(); } catch (IllegalArgumentException e) { throw new IllegalConfigurationException( - "Could not read " + TaskManagerOptions.MANAGED_MEMORY_SIZE.key(), e); + "Could not read " + TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.key(), e); } } else { // calculate managed memory size via fraction - final float fraction = config.getFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION); + final float fraction = config.getFloat(TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION); return (long) (fraction * heapAndManagedMemory); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java index 64f5a30bcb..ea6032f389 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java @@ -67,7 +67,7 @@ public class TaskManagerServicesConfiguration { /** * Managed memory (in megabytes). * - * @see TaskManagerOptions#MANAGED_MEMORY_SIZE + * @see TaskManagerOptions#LEGACY_MANAGED_MEMORY_SIZE */ private final long configuredMemory; @@ -201,7 +201,7 @@ public class TaskManagerServicesConfiguration { * * @return managed memory or a default value (currently -1) if not configured * - * @see TaskManagerOptions#MANAGED_MEMORY_SIZE + * @see TaskManagerOptions#LEGACY_MANAGED_MEMORY_SIZE */ long getConfiguredMemory() { return configuredMemory; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/NettyShuffleEnvironmentConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/NettyShuffleEnvironmentConfiguration.java index 6d4ca693bd..a6eb90f6f6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/NettyShuffleEnvironmentConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/NettyShuffleEnvironmentConfiguration.java @@ -231,8 +231,8 @@ public class NettyShuffleEnvironmentConfiguration { * *

The following configuration parameters are involved: *

    - *
  • {@link TaskManagerOptions#MANAGED_MEMORY_SIZE},
  • - *
  • {@link TaskManagerOptions#MANAGED_MEMORY_FRACTION},
  • + *
  • {@link TaskManagerOptions#LEGACY_MANAGED_MEMORY_SIZE},
  • + *
  • {@link TaskManagerOptions#LEGACY_MANAGED_MEMORY_FRACTION},
  • *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION},
  • *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN},
  • *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX}, and
  • diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ConfigurationParserUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ConfigurationParserUtils.java index 948d4c83c1..0d3788c09c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ConfigurationParserUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ConfigurationParserUtils.java @@ -43,21 +43,23 @@ public class ConfigurationParserUtils { */ public static long getManagedMemorySize(Configuration configuration) { long managedMemorySize; - String managedMemorySizeDefaultVal = TaskManagerOptions.MANAGED_MEMORY_SIZE.defaultValue(); - if (!configuration.getString(TaskManagerOptions.MANAGED_MEMORY_SIZE).equals(managedMemorySizeDefaultVal)) { + String managedMemorySizeDefaultVal = TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.defaultValue(); + if (!configuration.getString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE).equals(managedMemorySizeDefaultVal)) { try { managedMemorySize = MemorySize.parse( - configuration.getString(TaskManagerOptions.MANAGED_MEMORY_SIZE), MEGA_BYTES).getMebiBytes(); + configuration.getString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE), MEGA_BYTES).getMebiBytes(); } catch (IllegalArgumentException e) { - throw new IllegalConfigurationException("Could not read " + TaskManagerOptions.MANAGED_MEMORY_SIZE.key(), e); + throw new IllegalConfigurationException("Could not read " + TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE + + .key(), e); } } else { managedMemorySize = Long.valueOf(managedMemorySizeDefaultVal); } checkConfigParameter(configuration.getString( - TaskManagerOptions.MANAGED_MEMORY_SIZE).equals(TaskManagerOptions.MANAGED_MEMORY_SIZE.defaultValue()) || managedMemorySize > 0, - managedMemorySize, TaskManagerOptions.MANAGED_MEMORY_SIZE.key(), + TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE).equals(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.defaultValue()) || managedMemorySize > 0, + managedMemorySize, TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.key(), "MemoryManager needs at least one MB of memory. " + "If you leave this config parameter empty, the system automatically pick a fraction of the available memory."); @@ -71,10 +73,10 @@ public class ConfigurationParserUtils { * @return fraction of managed memory */ public static float getManagedMemoryFraction(Configuration configuration) { - float managedMemoryFraction = configuration.getFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION); + float managedMemoryFraction = configuration.getFloat(TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION); checkConfigParameter(managedMemoryFraction > 0.0f && managedMemoryFraction < 1.0f, managedMemoryFraction, - TaskManagerOptions.MANAGED_MEMORY_FRACTION.key(), + TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION.key(), "MemoryManager fraction of the free memory must be between 0.0 and 1.0"); return managedMemoryFraction; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactoryTest.java index 9a9468e599..ff61e65afd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactoryTest.java @@ -89,7 +89,7 @@ public class ActiveResourceManagerFactoryTest extends TestLogger { ClusterInformation clusterInformation, @Nullable String webInterfaceUrl, ResourceManagerMetricGroup resourceManagerMetricGroup) { - assertThat(configuration.contains(TaskManagerOptions.MANAGED_MEMORY_SIZE), is(true)); + assertThat(configuration.contains(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE), is(true)); return null; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java index cb020f690d..979860a1d6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java @@ -280,7 +280,7 @@ public class ResourceManagerTest extends TestLogger { @Test public void testCreateWorkerSlotProfiles() { final Configuration config = new Configuration(); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "100m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "100m"); config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 5); final ResourceProfile rmCalculatedResourceProfile = @@ -289,7 +289,7 @@ public class ResourceManagerTest extends TestLogger { final ResourceProfile tmCalculatedResourceProfile = TaskManagerServices.computeSlotResourceProfile( config.getInteger(TaskManagerOptions.NUM_TASK_SLOTS), - MemorySize.parse(config.getString(TaskManagerOptions.MANAGED_MEMORY_SIZE)).getBytes()); + MemorySize.parse(config.getString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE)).getBytes()); assertEquals(rmCalculatedResourceProfile, tmCalculatedResourceProfile); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NettyShuffleEnvironmentConfigurationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NettyShuffleEnvironmentConfigurationTest.java index aee4689858..a3e295f98e 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NettyShuffleEnvironmentConfigurationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NettyShuffleEnvironmentConfigurationTest.java @@ -247,15 +247,15 @@ public class NettyShuffleEnvironmentConfigurationTest extends TestLogger { config.setBoolean(TaskManagerOptions.MEMORY_OFF_HEAP, true); config.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "10m"); // 10MB + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "10m"); // 10MB assertEquals(890, TaskManagerServices.calculateHeapSizeMB(1000, config)); config.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.6f); assertEquals(390, TaskManagerServices.calculateHeapSizeMB(1000, config)); - config.removeConfig(TaskManagerOptions.MANAGED_MEMORY_SIZE); // use fraction of given memory + config.removeConfig(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE); // use fraction of given memory config.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); - config.setFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION, 0.1f); // 10% + config.setFloat(TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION, 0.1f); // 10% assertEquals(810, TaskManagerServices.calculateHeapSizeMB(1000, config)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java index ca83609a56..355784d378 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java @@ -46,32 +46,32 @@ public class NetworkBufferCalculationTest extends TestLogger { final long networkBufMax = 1L << 30; // 1GB config = getConfig( - Long.valueOf(TaskManagerOptions.MANAGED_MEMORY_SIZE.defaultValue()), - TaskManagerOptions.MANAGED_MEMORY_FRACTION.defaultValue(), + Long.valueOf(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.defaultValue()), + TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION.defaultValue(), 0.1f, networkBufMin, networkBufMax, MemoryType.HEAP); assertEquals(100L << 20, NettyShuffleEnvironmentConfiguration.calculateNewNetworkBufferMemory(config, 900L << 20)); // 900MB config = getConfig( - Long.valueOf(TaskManagerOptions.MANAGED_MEMORY_SIZE.defaultValue()), - TaskManagerOptions.MANAGED_MEMORY_FRACTION.defaultValue(), + Long.valueOf(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.defaultValue()), + TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION.defaultValue(), 0.2f, networkBufMin, networkBufMax, MemoryType.HEAP); assertEquals(200L << 20, NettyShuffleEnvironmentConfiguration.calculateNewNetworkBufferMemory(config, 800L << 20)); // 800MB config = getConfig( - Long.valueOf(TaskManagerOptions.MANAGED_MEMORY_SIZE.defaultValue()), - TaskManagerOptions.MANAGED_MEMORY_FRACTION.defaultValue(), + Long.valueOf(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.defaultValue()), + TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION.defaultValue(), 0.6f, networkBufMin, networkBufMax, MemoryType.HEAP); assertEquals(600L << 20, NettyShuffleEnvironmentConfiguration.calculateNewNetworkBufferMemory(config, 400L << 20)); // 400MB - config = getConfig(10, TaskManagerOptions.MANAGED_MEMORY_FRACTION.defaultValue(), + config = getConfig(10, TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION.defaultValue(), 0.1f, networkBufMin, networkBufMax, MemoryType.OFF_HEAP); assertEquals(100L << 20, NettyShuffleEnvironmentConfiguration.calculateNewNetworkBufferMemory(config, 890L << 20)); // 890MB - config = getConfig(10, TaskManagerOptions.MANAGED_MEMORY_FRACTION.defaultValue(), + config = getConfig(10, TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION.defaultValue(), 0.6f, networkBufMin, networkBufMax, MemoryType.OFF_HEAP); assertEquals(615L << 20, NettyShuffleEnvironmentConfiguration.calculateNewNetworkBufferMemory(config, 400L << 20)); // 400MB @@ -84,8 +84,8 @@ public class NetworkBufferCalculationTest extends TestLogger { /** * Returns a configuration for the tests. * - * @param managedMemory see {@link TaskManagerOptions#MANAGED_MEMORY_SIZE} - * @param managedMemoryFraction see {@link TaskManagerOptions#MANAGED_MEMORY_FRACTION} + * @param managedMemory see {@link TaskManagerOptions#LEGACY_MANAGED_MEMORY_SIZE} + * @param managedMemoryFraction see {@link TaskManagerOptions#LEGACY_MANAGED_MEMORY_FRACTION} * @param networkBufFraction see {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION} * @param networkBufMin see {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN} * @param networkBufMax see {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX} @@ -103,8 +103,8 @@ public class NetworkBufferCalculationTest extends TestLogger { final Configuration configuration = new Configuration(); - configuration.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE.key(), managedMemory); - configuration.setFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION.key(), managedMemoryFraction); + configuration.setLong(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.key(), managedMemory); + configuration.setFloat(TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION.key(), managedMemoryFraction); configuration.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.key(), networkBufFraction); configuration.setLong(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.key(), networkBufMin); configuration.setLong(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key(), networkBufMax); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerStartupTest.java index 58ef6a8f08..d2ace9562e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerStartupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerStartupTest.java @@ -123,7 +123,7 @@ public class TaskManagerRunnerStartupTest extends TestLogger { cfg.setBoolean(TaskManagerOptions.MANAGED_MEMORY_PRE_ALLOCATE, true); // something invalid - cfg.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "-42m"); + cfg.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "-42m"); try { startTaskManager( @@ -139,7 +139,7 @@ public class TaskManagerRunnerStartupTest extends TestLogger { // something ridiculously high final long memSize = (((long) Integer.MAX_VALUE - 1) * MemorySize.parse(TaskManagerOptions.MEMORY_SEGMENT_SIZE.defaultValue()).getBytes()) >> 20; - cfg.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, memSize + "m"); + cfg.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, memSize + "m"); try { startTaskManager( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/MiniClusterResource.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/MiniClusterResource.java index eb63ceb44a..b648c22f67 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/MiniClusterResource.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/MiniClusterResource.java @@ -159,8 +159,8 @@ public class MiniClusterResource extends ExternalResource { configuration.setBoolean(CoreOptions.FILESYTEM_DEFAULT_OVERRIDE, true); } - if (!configuration.contains(TaskManagerOptions.MANAGED_MEMORY_SIZE)) { - configuration.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, DEFAULT_MANAGED_MEMORY_SIZE); + if (!configuration.contains(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE)) { + configuration.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, DEFAULT_MANAGED_MEMORY_SIZE); } // set rest and rpc port to 0 to avoid clashes with concurrent MiniClusters diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java index b71006b31c..f655af62f0 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java @@ -88,7 +88,7 @@ public class LocalStreamEnvironment extends StreamExecutionEnvironment { Configuration configuration = new Configuration(); configuration.addAll(jobGraph.getJobConfiguration()); - configuration.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "0"); + configuration.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "0"); // add (and override) the settings with what the user defined configuration.addAll(this.configuration); diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java index c102acf8a4..24799e0478 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java @@ -119,7 +119,7 @@ public class LocalExecutorITCase extends TestLogger { private static Configuration getConfig() { Configuration config = new Configuration(); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "4m"); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); config.setBoolean(WebOptions.SUBMIT_ENABLE, false); diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/BatchAbstractTestBase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/BatchAbstractTestBase.java index 490b5dc17d..209e9da8f0 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/BatchAbstractTestBase.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/BatchAbstractTestBase.java @@ -47,7 +47,7 @@ public class BatchAbstractTestBase { private static Configuration getConfiguration() { Configuration config = new Configuration(); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "100m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "100m"); return config; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java index 8050dba491..d656184e78 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java @@ -58,7 +58,7 @@ public class AccumulatorErrorITCase extends TestLogger { public static Configuration getConfiguration() { Configuration config = new Configuration(); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "12m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "12m"); return config; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java index 490926fa63..c09943ff67 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java @@ -67,7 +67,7 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger { private static Configuration getConfiguration() { Configuration config = new Configuration(); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "48m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "48m"); config.setString(AkkaOptions.LOOKUP_TIMEOUT, "60 s"); config.setString(AkkaOptions.ASK_TIMEOUT, "60 s"); return config; diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java index 9eacee8afa..de681255c4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java @@ -209,7 +209,7 @@ public class EventTimeWindowCheckpointingITCase extends TestLogger { final File haDir = temporaryFolder.newFolder(); Configuration config = new Configuration(); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "48m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "48m"); // the default network buffers size (10% of heap max =~ 150MB) seems to much for this test case config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(80L << 20)); // 80 MB config.setString(AkkaOptions.FRAMESIZE, String.valueOf(MAX_MEM_STATE_SIZE) + "b"); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/KeyedStateCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/KeyedStateCheckpointingITCase.java index 0bc5023961..9ec23d3433 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/KeyedStateCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/KeyedStateCheckpointingITCase.java @@ -88,7 +88,7 @@ public class KeyedStateCheckpointingITCase extends TestLogger { private static Configuration getConfiguration() { Configuration config = new Configuration(); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "12m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "12m"); return config; } 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 37f0c33765..2f4a8572dd 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 @@ -664,7 +664,7 @@ public class SavepointITCase extends TestLogger { Configuration config = getFileBasedCheckpointsConfig(); config.addAll(jobGraph.getJobConfiguration()); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "0"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "0"); MiniClusterWithClientResource cluster = new MiniClusterWithClientResource( new MiniClusterResourceConfiguration.Builder() diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java index 8cce8009c9..c722311cff 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java @@ -80,7 +80,7 @@ public class WindowCheckpointingITCase extends TestLogger { private static Configuration getConfiguration() { Configuration config = new Configuration(); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "48m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "48m"); return config; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java index 7ecf4887e8..a09142e7ee 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java @@ -116,7 +116,7 @@ public class ClassLoaderITCase extends TestLogger { FOLDER.newFolder().getAbsoluteFile().toURI().toString()); // required as we otherwise run out of memory - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "80m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "80m"); miniClusterResource = new MiniClusterResource( new MiniClusterResourceConfiguration.Builder() diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java index 3cd7674037..032a3da85c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java @@ -64,7 +64,7 @@ public class JobSubmissionFailsITCase extends TestLogger { private static Configuration getConfiguration() { Configuration config = new Configuration(); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "4m"); return config; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java index 38902c7619..2398c14f8d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java +++ b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java @@ -50,7 +50,7 @@ public class StreamingScalabilityAndLatency { try { Configuration config = new Configuration(); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "80m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "80m"); config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 20000); config.setInteger("taskmanager.net.server.numThreads", 1); diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java index a1e4add6a0..0177e4a9b1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java @@ -59,7 +59,7 @@ public class CustomSerializationITCase extends TestLogger { public static Configuration getConfiguration() { Configuration config = new Configuration(); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "30m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "30m"); return config; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java index c61657dc9b..e9523f90ec 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java @@ -62,7 +62,7 @@ public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger { private static Configuration getConfiguration() { Configuration config = new Configuration(); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "80m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "80m"); config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 800); return config; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java index 1677a88a3a..c8f44e8a27 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java @@ -102,7 +102,7 @@ public abstract class AbstractTaskManagerProcessFailureRecoveryTest extends Test config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperResource.getConnectString()); config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getAbsolutePath()); config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "4m"); config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 100); try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = new StandaloneSessionClusterEntrypoint(config)) { 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 b7c91cbb75..130d38ef21 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 @@ -249,7 +249,7 @@ public class JobManagerHAProcessFailureRecoveryITCase extends TestLogger { Configuration config = ZooKeeperTestUtils.createZooKeeperHAConfig( zooKeeper.getConnectString(), zookeeperStoragePath.getPath()); // Task manager configuration - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "4m"); config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 100); config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); 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 3b0fa172fc..bd76b0942a 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 @@ -115,7 +115,7 @@ public class ProcessFailureCancelingITCase extends TestLogger { config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperResource.getConnectString()); config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getAbsolutePath()); config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "4m"); config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 100); config.setInteger(RestOptions.PORT, 0); diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java index c312b04410..8fe2aa3518 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java @@ -77,7 +77,7 @@ public class IPv6HostnamesITCase extends TestLogger { Configuration config = new Configuration(); config.setString(JobManagerOptions.ADDRESS, addressString); config.setString(TaskManagerOptions.HOST, addressString); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "16m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "16m"); return config; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java index 24effd746c..f0938a2458 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java @@ -89,7 +89,7 @@ public class TimestampITCase extends TestLogger { private static Configuration getConfiguration() { Configuration config = new Configuration(); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "12m"); + config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "12m"); return config; } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java index 563d646b7e..7459bac030 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java @@ -218,6 +218,6 @@ public class YarnConfigurationITCase extends YarnTestBase { private static int calculateManagedMemorySizeMB(Configuration configuration) { Configuration resourceManagerConfig = ActiveResourceManagerFactory.createActiveResourceManagerConfiguration(configuration); - return MemorySize.parse(resourceManagerConfig.getString(TaskManagerOptions.MANAGED_MEMORY_SIZE)).getMebiBytes(); + return MemorySize.parse(resourceManagerConfig.getString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE)).getMebiBytes(); } } -- Gitee From 2f944ba3c71d30f289f60306222f566315fb6039 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 23 Sep 2019 11:19:14 +0800 Subject: [PATCH 208/268] [FLINK-13982][runtime] Introduce FLIP-49 task executor memory config options. --- docs/_includes/generated/common_section.html | 6 +- ...tty_shuffle_environment_configuration.html | 15 -- .../generated/task_manager_configuration.html | 5 - .../task_manager_memory_configuration.html | 83 +++++++- .../NettyShuffleEnvironmentOptions.java | 13 +- .../configuration/TaskManagerOptions.java | 188 +++++++++++++++++- 6 files changed, 275 insertions(+), 35 deletions(-) diff --git a/docs/_includes/generated/common_section.html b/docs/_includes/generated/common_section.html index 1c6685b2f4..8d7a232446 100644 --- a/docs/_includes/generated/common_section.html +++ b/docs/_includes/generated/common_section.html @@ -13,9 +13,9 @@ JVM heap size for the JobManager. -
    taskmanager.heap.size
    - "1024m" - JVM heap size for the TaskManagers, which are the parallel workers of the system. On YARN setups, this value is automatically configured to the size of the TaskManager's YARN container, minus a certain tolerance value. +
    taskmanager.memory.total-flink.size
    + (none) + Total Flink Memory size for the TaskExecutors. This includes all the memory that a TaskExecutor consumes, except for JVM Metaspace and JVM Overhead. It consists of Framework Heap Memory, Task Heap Memory, Task Off-Heap Memory, Managed Memory, and Shuffle Memory.
    parallelism.default
    diff --git a/docs/_includes/generated/netty_shuffle_environment_configuration.html b/docs/_includes/generated/netty_shuffle_environment_configuration.html index 7787ed0d93..5a4f3a8462 100644 --- a/docs/_includes/generated/netty_shuffle_environment_configuration.html +++ b/docs/_includes/generated/netty_shuffle_environment_configuration.html @@ -32,21 +32,6 @@ 8 Number of extra network buffers to use for each outgoing/incoming gate (result partition/input gate). In credit-based flow control mode, this indicates how many floating credits are shared among all the input channels. The floating buffers are distributed based on backlog (real-time output buffers in the subpartition) feedback, and can help relieve back-pressure caused by unbalanced data distribution among the subpartitions. This value should be increased in case of higher round trip times between nodes and/or larger number of machines in the cluster. - -
    taskmanager.network.memory.fraction
    - 0.1 - Fraction of JVM memory to use for network buffers. This determines how many streaming data exchange channels a TaskManager can have at the same time and how well buffered the channels are. If a job is rejected or you get a warning that the system has not enough buffers available, increase this value or the min/max values below. Also note, that "taskmanager.network.memory.min"` and "taskmanager.network.memory.max" may override this fraction. - - -
    taskmanager.network.memory.max
    - "1gb" - Maximum memory size for network buffers. - - -
    taskmanager.network.memory.min
    - "64mb" - Minimum memory size for network buffers. -
    taskmanager.network.request-backoff.initial
    100 diff --git a/docs/_includes/generated/task_manager_configuration.html b/docs/_includes/generated/task_manager_configuration.html index 16da6d52cd..ca649b526a 100644 --- a/docs/_includes/generated/task_manager_configuration.html +++ b/docs/_includes/generated/task_manager_configuration.html @@ -42,11 +42,6 @@ false Whether the quarantine monitor for task managers shall be started. The quarantine monitor shuts down the actor system if it detects that it has quarantined another actor system or if it has been quarantined by another actor system. - -
    taskmanager.heap.size
    - "1024m" - JVM heap size for the TaskManagers, which are the parallel workers of the system. On YARN setups, this value is automatically configured to the size of the TaskManager's YARN container, minus a certain tolerance value. -
    taskmanager.host
    (none) diff --git a/docs/_includes/generated/task_manager_memory_configuration.html b/docs/_includes/generated/task_manager_memory_configuration.html index c8433ed9e6..e05115d4b4 100644 --- a/docs/_includes/generated/task_manager_memory_configuration.html +++ b/docs/_includes/generated/task_manager_memory_configuration.html @@ -8,14 +8,49 @@ -
    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 (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.framework.heap.size
    + "128m" + Framework Heap Memory size for TaskExecutors. This is the size of JVM heap memory reserved for TaskExecutor framework, which will not be allocated to task slots. -
    taskmanager.memory.off-heap
    - false - 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.
    When set to true, then it is advised that `taskmanager.memory.preallocate` is also set to true. +
    taskmanager.memory.jvm-metaspace.size
    + "192m" + JVM Metaspace Size for the TaskExecutors. + + +
    taskmanager.memory.jvm-overhead.fraction
    + 0.1 + Fraction of Total Process Memory to be reserved for JVM Overhead. This is off-heap memory reserved for JVM overhead, such as thread stack space, I/O direct memory, compile cache, etc. The size of JVM Overhead is derived to make up the configured fraction of the Total Process Memory. If the derived size is less/greater than the configured min/max size, the min/max size will be used. The exact size of JVM Overhead can be explicitly specified by setting the min/max size to the same value. + + +
    taskmanager.memory.jvm-overhead.max
    + "1g" + Max JVM Overhead size for the TaskExecutors. This is off-heap memory reserved for JVM overhead, such as thread stack space, I/O direct memory, compile cache, etc. The size of JVM Overhead is derived to make up the configured fraction of the Total Process Memory. If the derived size is less/greater than the configured min/max size, the min/max size will be used. The exact size of JVM Overhead can be explicitly specified by setting the min/max size to the same value. + + +
    taskmanager.memory.jvm-overhead.min
    + "128m" + Min JVM Overhead size for the TaskExecutors. This is off-heap memory reserved for JVM overhead, such as thread stack space, I/O direct memory, compile cache, etc. The size of JVM Overhead is derived to make up the configured fraction of the Total Process Memory. If the derived size is less/greater than the configured min/max size, the min/max size will be used. The exact size of JVM Overhead can be explicitly specified by setting the min/max size to the same value. + + +
    taskmanager.memory.managed.fraction
    + 0.5 + Fraction of Total Flink Memory to be used as Managed Memory, if Managed Memory size is not explicitly specified. + + +
    taskmanager.memory.managed.off-heap.fraction
    + -1.0 + Fraction of Managed Memory that Off-Heap Managed Memory takes, if Off-Heap Managed Memory size is not explicitly specified. If the fraction is not explicitly specified (or configured with negative values), it will be derived from the legacy config option 'taskmanager.memory.off-heap', to use either all on-heap memory or all off-heap memory for Managed Memory. + + +
    taskmanager.memory.managed.off-heap.size
    + (none) + Off-Heap Managed Memory size for TaskExecutors. This is the part of Managed Memory that is off-heap, while the remaining is on-heap. If unspecified, it will be derived to make up the configured fraction of the Managed Memory size. + + +
    taskmanager.memory.managed.size
    + (none) + Managed Memory size for TaskExecutors. This is the size of memory managed by the memory manager, including both On-Heap Managed Memory and Off-Heap Managed Memory, reserved for sorting, hash tables, caching of intermediate results and state backends. Memory consumers can either allocate memory from the memory manager in the form of MemorySegments, or reserve bytes from the memory manager and keep their memory usage within that boundary. If unspecified, it will be derived to make up the configured fraction of the Total Flink Memory.
    taskmanager.memory.preallocate
    @@ -28,9 +63,39 @@ 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. +
    taskmanager.memory.shuffle.fraction
    + 0.1 + Fraction of Total Flink Memory to be used as Shuffle Memory. Shuffle Memory is off-heap memory reserved for ShuffleEnvironment (e.g., network buffers). Shuffle Memory size is derived to make up the configured fraction of the Total Flink Memory. If the derived size is less/greater than the configured min/max size, the min/max size will be used. The exact size of Shuffle Memory can be explicitly specified by setting the min/max size to the same value. + + +
    taskmanager.memory.shuffle.max
    + "1g" + Max Shuffle Memory size for TaskExecutors. Shuffle Memory is off-heap memory reserved for ShuffleEnvironment (e.g., network buffers). Shuffle Memory size is derived to make up the configured fraction of the Total Flink Memory. If the derived size is less/greater than the configured min/max size, the min/max size will be used. The exact size of Shuffle Memory can be explicitly specified by setting the min/max to the same value. + + +
    taskmanager.memory.shuffle.min
    + "64m" + Min Shuffle Memory size for TaskExecutors. Shuffle Memory is off-heap memory reserved for ShuffleEnvironment (e.g., network buffers). Shuffle Memory size is derived to make up the configured fraction of the Total Flink Memory. If the derived size is less/greater than the configured min/max size, the min/max size will be used. The exact size of Shuffle Memory can be explicitly specified by setting the min/max to the same value. + + +
    taskmanager.memory.task.heap.size
    + (none) + Task Heap Memory size for TaskExecutors. This is the size of JVM heap memory reserved for user code. If not specified, it will be derived as Total Flink Memory minus Framework Heap Memory, Task Off-Heap Memory, (On-Heap and Off-Heap) Managed Memory and Shuffle Memory. + + +
    taskmanager.memory.task.off-heap.size
    + "0b" + Task Heap Memory size for TaskExecutors. This is the size of off heap memory (JVM direct memory or native memory) reserved for user code. + + +
    taskmanager.memory.total-flink.size
    + (none) + Total Flink Memory size for the TaskExecutors. This includes all the memory that a TaskExecutor consumes, except for JVM Metaspace and JVM Overhead. It consists of Framework Heap Memory, Task Heap Memory, Task Off-Heap Memory, Managed Memory, and Shuffle Memory. + + +
    taskmanager.memory.total-process.size
    + (none) + Total Process Memory size for the TaskExecutors. This includes all the memory that a TaskExecutor consumes, consisting of Total Flink Memory, JVM Metaspace, and JVM Overhead. On containerized setups, this should be set to the container memory. diff --git a/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java index 733085ebce..4e2e956225 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java @@ -80,8 +80,8 @@ public class NettyShuffleEnvironmentOptions { * Number of buffers used in the network stack. This defines the number of possible tasks and * shuffles. * - * @deprecated use {@link #NETWORK_BUFFERS_MEMORY_FRACTION}, {@link #NETWORK_BUFFERS_MEMORY_MIN}, - * and {@link #NETWORK_BUFFERS_MEMORY_MAX} instead + * @deprecated use {@link TaskManagerOptions#SHUFFLE_MEMORY_FRACTION}, {@link TaskManagerOptions#SHUFFLE_MEMORY_MIN}, + * and {@link TaskManagerOptions#SHUFFLE_MEMORY_MAX} instead */ @Deprecated public static final ConfigOption NETWORK_NUM_BUFFERS = @@ -90,7 +90,10 @@ public class NettyShuffleEnvironmentOptions { /** * Fraction of JVM memory to use for network buffers. + * + * @deprecated use {@link TaskManagerOptions#SHUFFLE_MEMORY_FRACTION} instead */ + @Deprecated public static final ConfigOption NETWORK_BUFFERS_MEMORY_FRACTION = key("taskmanager.network.memory.fraction") .defaultValue(0.1f) @@ -102,7 +105,10 @@ public class NettyShuffleEnvironmentOptions { /** * Minimum memory size for network buffers. + * + * @deprecated use {@link TaskManagerOptions#SHUFFLE_MEMORY_MIN} instead */ + @Deprecated public static final ConfigOption NETWORK_BUFFERS_MEMORY_MIN = key("taskmanager.network.memory.min") .defaultValue("64mb") @@ -110,7 +116,10 @@ public class NettyShuffleEnvironmentOptions { /** * Maximum memory size for network buffers. + * + * @deprecated use {@link TaskManagerOptions#SHUFFLE_MEMORY_MAX} instead */ + @Deprecated public static final ConfigOption NETWORK_BUFFERS_MEMORY_MAX = key("taskmanager.network.memory.max") .defaultValue("1gb") 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 acab022b43..57e4d0363e 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 @@ -44,7 +44,7 @@ public class TaskManagerOptions { /** * JVM heap size for the TaskManagers with memory size. */ - @Documentation.CommonOption(position = Documentation.CommonOption.POSITION_MEMORY) + @Deprecated public static final ConfigOption TASK_MANAGER_HEAP_MEMORY = key("taskmanager.heap.size") .defaultValue("1024m") @@ -191,6 +191,7 @@ public class TaskManagerOptions { * Amount of memory to be allocated by the task manager's memory manager. If not * set, a relative fraction will be allocated, as defined by {@link #LEGACY_MANAGED_MEMORY_FRACTION}. */ + @Deprecated public static final ConfigOption LEGACY_MANAGED_MEMORY_SIZE = key("taskmanager.memory.size") .defaultValue("0") @@ -203,6 +204,7 @@ public class TaskManagerOptions { * Fraction of free memory allocated by the memory manager if {@link #LEGACY_MANAGED_MEMORY_SIZE} is * not set. */ + @Deprecated public static final ConfigOption LEGACY_MANAGED_MEMORY_FRACTION = key("taskmanager.memory.fraction") .defaultValue(0.7f) @@ -220,6 +222,7 @@ public class TaskManagerOptions { * Memory allocation method (JVM heap or off-heap), used for managed memory of the TaskManager * as well as the network buffers. **/ + @Deprecated public static final ConfigOption MEMORY_OFF_HEAP = key("taskmanager.memory.off-heap") .defaultValue(false) @@ -260,6 +263,189 @@ public class TaskManagerOptions { text("\"ip\" - uses host's ip address as binding address")) .build()); + // ------------------------------------------------------------------------ + // Memory Options + // ------------------------------------------------------------------------ + + /** + * Total Process Memory size for the TaskExecutors. + */ + public static final ConfigOption TOTAL_PROCESS_MEMORY = + key("taskmanager.memory.total-process.size") + .noDefaultValue() + .withDescription("Total Process Memory size for the TaskExecutors. This includes all the memory that a" + + " TaskExecutor consumes, consisting of Total Flink Memory, JVM Metaspace, and JVM Overhead. On" + + " containerized setups, this should be set to the container memory."); + + /** + * Total Flink Memory size for the TaskExecutors. + */ + @Documentation.CommonOption(position = Documentation.CommonOption.POSITION_MEMORY) + public static final ConfigOption TOTAL_FLINK_MEMORY = + key("taskmanager.memory.total-flink.size") + .noDefaultValue() + .withDeprecatedKeys(TASK_MANAGER_HEAP_MEMORY.key()) + .withDescription("Total Flink Memory size for the TaskExecutors. This includes all the memory that a" + + " TaskExecutor consumes, except for JVM Metaspace and JVM Overhead. It consists of Framework Heap Memory," + + " Task Heap Memory, Task Off-Heap Memory, Managed Memory, and Shuffle Memory."); + + /** + * Framework Heap Memory size for TaskExecutors. + */ + public static final ConfigOption FRAMEWORK_HEAP_MEMORY = + key("taskmanager.memory.framework.heap.size") + .defaultValue("128m") + .withDescription("Framework Heap Memory size for TaskExecutors. This is the size of JVM heap memory reserved" + + " for TaskExecutor framework, which will not be allocated to task slots."); + + /** + * Task Heap Memory size for TaskExecutors. + */ + public static final ConfigOption TASK_HEAP_MEMORY = + key("taskmanager.memory.task.heap.size") + .noDefaultValue() + .withDescription("Task Heap Memory size for TaskExecutors. This is the size of JVM heap memory reserved for" + + " user code. If not specified, it will be derived as Total Flink Memory minus Framework Heap Memory," + + " Task Off-Heap Memory, (On-Heap and Off-Heap) Managed Memory and Shuffle Memory."); + + /** + * Task Off-Heap Memory size for TaskExecutors. + */ + public static final ConfigOption TASK_OFF_HEAP_MEMORY = + key("taskmanager.memory.task.off-heap.size") + .defaultValue("0b") + .withDescription("Task Heap Memory size for TaskExecutors. This is the size of off heap memory (JVM direct" + + " memory or native memory) reserved for user code."); + + /** + * Managed Memory size for TaskExecutors. + */ + public static final ConfigOption MANAGED_MEMORY_SIZE = + key("taskmanager.memory.managed.size") + .noDefaultValue() + .withDeprecatedKeys(LEGACY_MANAGED_MEMORY_SIZE.key()) + .withDescription("Managed Memory size for TaskExecutors. This is the size of memory managed by the memory" + + " manager, including both On-Heap Managed Memory and Off-Heap Managed Memory, reserved for sorting," + + " hash tables, caching of intermediate results and state backends. Memory consumers can either" + + " allocate memory from the memory manager in the form of MemorySegments, or reserve bytes from the" + + " memory manager and keep their memory usage within that boundary. If unspecified, it will be derived" + + " to make up the configured fraction of the Total Flink Memory."); + + /** + * Fraction of Total Flink Memory to be used as Managed Memory, if {@link #MANAGED_MEMORY_SIZE} is not specified. + */ + public static final ConfigOption MANAGED_MEMORY_FRACTION = + key("taskmanager.memory.managed.fraction") + .defaultValue(0.5f) + .withDescription("Fraction of Total Flink Memory to be used as Managed Memory, if Managed Memory size is not" + + " explicitly specified."); + + /** + * Off-Heap Managed Memory size for TaskExecutors. + */ + public static final ConfigOption MANAGED_MEMORY_OFFHEAP_SIZE = + key("taskmanager.memory.managed.off-heap.size") + .noDefaultValue() + .withDescription("Off-Heap Managed Memory size for TaskExecutors. This is the part of Managed Memory that is" + + " off-heap, while the remaining is on-heap. If unspecified, it will be derived to make up the" + + " configured fraction of the Managed Memory size."); + + /** + * Fraction of Managed Memory that Off-Heap Managed Memory takes. + */ + public static final ConfigOption MANAGED_MEMORY_OFFHEAP_FRACTION = + key("taskmanager.memory.managed.off-heap.fraction") + .defaultValue(-1.0f) + .withDescription("Fraction of Managed Memory that Off-Heap Managed Memory takes, if Off-Heap Managed Memory" + + " size is not explicitly specified. If the fraction is not explicitly specified (or configured with" + + " negative values), it will be derived from the legacy config option '" + + TaskManagerOptions.MEMORY_OFF_HEAP.key() + "', to use either all on-heap memory or all off-heap memory" + + " for Managed Memory."); + + /** + * Min Shuffle Memory size for TaskExecutors. + */ + public static final ConfigOption SHUFFLE_MEMORY_MIN = + key("taskmanager.memory.shuffle.min") + .defaultValue("64m") + .withDeprecatedKeys(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.key()) + .withDescription("Min Shuffle Memory size for TaskExecutors. Shuffle Memory is off-heap memory reserved for" + + " ShuffleEnvironment (e.g., network buffers). Shuffle Memory size is derived to make up the configured" + + " fraction of the Total Flink Memory. If the derived size is less/greater than the configured min/max" + + " size, the min/max size will be used. The exact size of Shuffle Memory can be explicitly specified by" + + " setting the min/max to the same value."); + + /** + * Max Shuffle Memory size for TaskExecutors. + */ + public static final ConfigOption SHUFFLE_MEMORY_MAX = + key("taskmanager.memory.shuffle.max") + .defaultValue("1g") + .withDeprecatedKeys(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key()) + .withDescription("Max Shuffle Memory size for TaskExecutors. Shuffle Memory is off-heap memory reserved for" + + " ShuffleEnvironment (e.g., network buffers). Shuffle Memory size is derived to make up the configured" + + " fraction of the Total Flink Memory. If the derived size is less/greater than the configured min/max" + + " size, the min/max size will be used. The exact size of Shuffle Memory can be explicitly specified by" + + " setting the min/max to the same value."); + + /** + * Fraction of Total Flink Memory to be used as Shuffle Memory. + */ + public static final ConfigOption SHUFFLE_MEMORY_FRACTION = + key("taskmanager.memory.shuffle.fraction") + .defaultValue(0.1f) + .withDeprecatedKeys(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.key()) + .withDescription("Fraction of Total Flink Memory to be used as Shuffle Memory. Shuffle Memory is off-heap" + + " memory reserved for ShuffleEnvironment (e.g., network buffers). Shuffle Memory size is derived to" + + " make up the configured fraction of the Total Flink Memory. If the derived size is less/greater than" + + " the configured min/max size, the min/max size will be used. The exact size of Shuffle Memory can be" + + " explicitly specified by setting the min/max size to the same value."); + + /** + * JVM Metaspace Size for the TaskExecutors. + */ + public static final ConfigOption JVM_METASPACE = + key("taskmanager.memory.jvm-metaspace.size") + .defaultValue("192m") + .withDescription("JVM Metaspace Size for the TaskExecutors."); + + /** + * Min JVM Overhead size for the TaskExecutors. + */ + public static final ConfigOption JVM_OVERHEAD_MIN = + key("taskmanager.memory.jvm-overhead.min") + .defaultValue("128m") + .withDescription("Min JVM Overhead size for the TaskExecutors. This is off-heap memory reserved for JVM" + + " overhead, such as thread stack space, I/O direct memory, compile cache, etc. The size of JVM" + + " Overhead is derived to make up the configured fraction of the Total Process Memory. If the derived" + + " size is less/greater than the configured min/max size, the min/max size will be used. The exact size" + + " of JVM Overhead can be explicitly specified by setting the min/max size to the same value."); + + /** + * Max JVM Overhead size for the TaskExecutors. + */ + public static final ConfigOption JVM_OVERHEAD_MAX = + key("taskmanager.memory.jvm-overhead.max") + .defaultValue("1g") + .withDescription("Max JVM Overhead size for the TaskExecutors. This is off-heap memory reserved for JVM" + + " overhead, such as thread stack space, I/O direct memory, compile cache, etc. The size of JVM" + + " Overhead is derived to make up the configured fraction of the Total Process Memory. If the derived" + + " size is less/greater than the configured min/max size, the min/max size will be used. The exact size" + + " of JVM Overhead can be explicitly specified by setting the min/max size to the same value."); + + /** + * Fraction of Total Process Memory to be reserved for JVM Overhead. + */ + public static final ConfigOption JVM_OVERHEAD_FRACTION = + key("taskmanager.memory.jvm-overhead.fraction") + .defaultValue(0.1f) + .withDescription("Fraction of Total Process Memory to be reserved for JVM Overhead. This is off-heap memory" + + " reserved for JVM overhead, such as thread stack space, I/O direct memory, compile cache, etc. The" + + " size of JVM Overhead is derived to make up the configured fraction of the Total Process Memory. If" + + " the derived size is less/greater than the configured min/max size, the min/max size will be used." + + " The exact size of JVM Overhead can be explicitly specified by setting the min/max size to the same" + + " value."); + // ------------------------------------------------------------------------ // Task Options // ------------------------------------------------------------------------ -- Gitee From 9b14f93eeb856c98c980fb40337ee74c488d3973 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 23 Sep 2019 16:20:39 +0800 Subject: [PATCH 209/268] [FLINK-13982][core] Introduce arithmetic operations (add, subtract, multiply) for MemorySize. --- .../flink/configuration/MemorySize.java | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/MemorySize.java b/flink-core/src/main/java/org/apache/flink/configuration/MemorySize.java index 092c0128af..5d9e22d2a8 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/MemorySize.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/MemorySize.java @@ -20,6 +20,7 @@ package org.apache.flink.configuration; import org.apache.flink.annotation.PublicEvolving; +import java.math.BigDecimal; import java.util.Locale; import static org.apache.flink.configuration.MemorySize.MemoryUnit.BYTES; @@ -115,6 +116,28 @@ public class MemorySize implements java.io.Serializable { return bytes + " bytes"; } + // ------------------------------------------------------------------------ + // Calculations + // ------------------------------------------------------------------------ + + public MemorySize add(MemorySize that) { + return new MemorySize(Math.addExact(this.bytes, that.bytes)); + } + + public MemorySize subtract(MemorySize that) { + return new MemorySize(Math.subtractExact(this.bytes, that.bytes)); + } + + public MemorySize multiply(double multiplier) { + checkArgument(multiplier >= 0, "multiplier must be >= 0"); + + BigDecimal product = BigDecimal.valueOf(this.bytes).multiply(BigDecimal.valueOf(multiplier)); + if (product.compareTo(BigDecimal.valueOf(Long.MAX_VALUE)) > 0) { + throw new ArithmeticException("long overflow"); + } + return new MemorySize(product.longValue()); + } + // ------------------------------------------------------------------------ // Parsing // ------------------------------------------------------------------------ -- Gitee From cf337043236c852eaa16e4e51d6c4e95d9a6d056 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 23 Sep 2019 16:28:12 +0800 Subject: [PATCH 210/268] [FLINK-13982][runtime] Introduce 'TaskExecutorResourceSpec' to store memory / pool sizes of TaskExecutors. --- .../TaskExecutorResourceSpec.java | 158 ++++++++++++++++++ 1 file changed, 158 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceSpec.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceSpec.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceSpec.java new file mode 100644 index 0000000000..0813aead4c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceSpec.java @@ -0,0 +1,158 @@ +/* + * 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.clusterframework; + +import org.apache.flink.configuration.MemorySize; + +/** + * Describe the specifics of different resource dimensions of the TaskExecutor. + * + *

    A TaskExecutor's memory consists of the following components. + *

      + *
    • Framework Heap Memory
    • + *
    • Task Heap Memory
    • + *
    • Task Off-Heap Memory
    • + *
    • Shuffle Memory
    • + *
    • Managed Memory
    • + *
        + *
      • On-Heap Managed Memory
      • + *
      • Off-Heap Managed Memory
      • + *
      + *
    • JVM Metaspace
    • + *
    • JVM Overhead
    • + *
    + * Among all the components, Framework Heap Memory, Task Heap Memory and On-Heap Managed Memory use on heap memory, + * while the rest use off heap memory. We use Total Process Memory to refer to all the memory components, while Total + * Flink Memory refering to all the components except JVM Metaspace and JVM Overhead. + * + *

    The relationships of TaskExecutor memory components are shown below. + *

    + *               ┌ ─ ─ Total Process Memory  ─ ─ ┐
    + *                ┌ ─ ─ Total Flink Memory  ─ ─ ┐
    + *               │ ┌───────────────────────────┐ │
    + *                ││   Framework Heap Memory   ││  ─┐
    + *               │ └───────────────────────────┘ │  │
    + *                │┌───────────────────────────┐│   │
    + *               │ │     Task Heap Memory      │ │ ─┤
    + *                │└───────────────────────────┘│   │
    + *               │ ┌───────────────────────────┐ │  │
    + *            ┌─  ││   Task Off-Heap Memory    ││   │
    + *            │  │ └───────────────────────────┘ │  ├─ On-Heap
    + *            │   │┌───────────────────────────┐│   │
    + *            ├─ │ │      Shuffle Memory       │ │  │
    + *            │   │└───────────────────────────┘│   │
    + *            │  │ ┌───── Managed Memory ──────┐ │  │
    + *            │   ││┌─────────────────────────┐││   │
    + *            │  │ ││ On-Heap Managed Memory  ││ │ ─┘
    + *            │   ││├─────────────────────────┤││
    + *  Off-Heap ─┼─ │ ││ Off-Heap Managed Memory ││ │
    + *            │   ││└─────────────────────────┘││
    + *            │  │ └───────────────────────────┘ │
    + *            │   └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
    + *            │  │┌─────────────────────────────┐│
    + *            ├─  │        JVM Metaspace        │
    + *            │  │└─────────────────────────────┘│
    + *            │   ┌─────────────────────────────┐
    + *            └─ ││        JVM Overhead         ││
    + *                └─────────────────────────────┘
    + *               └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
    + * 
    + */ +public class TaskExecutorResourceSpec { + + private final MemorySize frameworkHeapSize; + + private final MemorySize taskHeapSize; + + private final MemorySize taskOffHeapSize; + + private final MemorySize shuffleMemSize; + + private final MemorySize onHeapManagedMemorySize; + + private final MemorySize offHeapManagedMemorySize; + + private final MemorySize jvmMetaspaceSize; + + private final MemorySize jvmOverheadSize; + + public TaskExecutorResourceSpec( + MemorySize frameworkHeapSize, + MemorySize taskHeapSize, + MemorySize taskOffHeapSize, + MemorySize shuffleMemSize, + MemorySize onHeapManagedMemorySize, + MemorySize offHeapManagedMemorySize, + MemorySize jvmMetaspaceSize, + MemorySize jvmOverheadSize) { + + this.frameworkHeapSize = frameworkHeapSize; + this.taskHeapSize = taskHeapSize; + this.taskOffHeapSize = taskOffHeapSize; + this.shuffleMemSize = shuffleMemSize; + this.onHeapManagedMemorySize = onHeapManagedMemorySize; + this.offHeapManagedMemorySize = offHeapManagedMemorySize; + this.jvmMetaspaceSize = jvmMetaspaceSize; + this.jvmOverheadSize = jvmOverheadSize; + } + + public MemorySize getFrameworkHeapSize() { + return frameworkHeapSize; + } + + public MemorySize getTaskHeapSize() { + return taskHeapSize; + } + + public MemorySize getTaskOffHeapSize() { + return taskOffHeapSize; + } + + public MemorySize getShuffleMemSize() { + return shuffleMemSize; + } + + public MemorySize getOnHeapManagedMemorySize() { + return onHeapManagedMemorySize; + } + + public MemorySize getOffHeapManagedMemorySize() { + return offHeapManagedMemorySize; + } + + public MemorySize getManagedMemorySize() { + return onHeapManagedMemorySize.add(offHeapManagedMemorySize); + } + + public MemorySize getJvmMetaspaceSize() { + return jvmMetaspaceSize; + } + + public MemorySize getJvmOverheadSize() { + return jvmOverheadSize; + } + + public MemorySize getTotalFlinkMemorySize() { + return frameworkHeapSize.add(taskHeapSize).add(taskOffHeapSize).add(shuffleMemSize).add(getManagedMemorySize()); + } + + public MemorySize getTotalProcessMemorySize() { + return getTotalFlinkMemorySize().add(jvmMetaspaceSize).add(jvmOverheadSize); + } +} -- Gitee From 59eca61bff3965a71cdb16865050c3daa0c8014b Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Tue, 24 Sep 2019 11:32:51 +0800 Subject: [PATCH 211/268] [FLINK-13982][runtime] Introduce 'TaskExecutorResourceUtils' for calculating memory / pool sizes from configuration. --- .../TaskExecutorResourceUtils.java | 565 +++++++++++++++ .../TaskExecutorResourceUtilsTest.java | 655 ++++++++++++++++++ 2 files changed, 1220 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtils.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtilsTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtils.java new file mode 100644 index 0000000000..ba521bb10c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtils.java @@ -0,0 +1,565 @@ +/* + * 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.clusterframework; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.util.ConfigurationParserUtils; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Utility class for TaskExecutor memory configurations. + * + *

    See {@link TaskExecutorResourceSpec} for details about memory components of TaskExecutor and their relationships. + */ +public class TaskExecutorResourceUtils { + + private TaskExecutorResourceUtils() {} + + // ------------------------------------------------------------------------ + // Memory Configuration Calculations + // ------------------------------------------------------------------------ + + public static TaskExecutorResourceSpec resourceSpecFromConfig(final Configuration config) { + if (isTaskHeapMemorySizeExplicitlyConfigured(config) && isManagedMemorySizeExplicitlyConfigured(config)) { + // both task heap memory and managed memory are configured, use these to derive total flink memory + return deriveResourceSpecWithExplicitTaskAndManagedMemory(config); + } else if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) { + // either of task heap memory and managed memory is not configured, total flink memory is configured, + // derive from total flink memory + return deriveResourceSpecWithTotalFlinkMemory(config); + } else if (isTotalProcessMemorySizeExplicitlyConfigured(config)) { + // total flink memory is not configured, total process memory is configured, + // derive from total process memory + return deriveResourceSpecWithTotalProcessMemory(config); + } else { + throw new IllegalConfigurationException("Either Task Heap Memory size and Managed Memory size, or Total Flink" + + " Memory size, or Total Process Memory size need to be configured explicitly."); + } + } + + private static TaskExecutorResourceSpec deriveResourceSpecWithExplicitTaskAndManagedMemory(final Configuration config) { + // derive flink internal memory from explicitly configure task heap memory size and managed memory size + + final MemorySize taskHeapMemorySize = getTaskHeapMemorySize(config); + final MemorySize managedMemorySize = getManagedMemorySize(config); + + final MemorySize frameworkHeapMemorySize = getFrameworkHeapMemorySize(config); + final MemorySize taskOffHeapMemorySize = getTaskOffHeapMemorySize(config); + + final OnHeapAndOffHeapManagedMemory onHeapAndOffHeapManagedMemory = deriveOnHeapAndOffHeapMemoryFromManagedMemory(config, managedMemorySize); + + final MemorySize shuffleMemorySize; + final MemorySize totalFlinkExcludeShuffleMemorySize = + frameworkHeapMemorySize.add(taskHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize); + + if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) { + // derive shuffle memory from total flink memory, and check against shuffle min/max + final MemorySize totalFlinkMemorySize = getTotalFlinkMemorySize(config); + if (totalFlinkExcludeShuffleMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { + throw new IllegalConfigurationException( + "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toString() + + "), Task Heap Memory (" + taskHeapMemorySize.toString() + + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toString() + + ") and Managed Memory (" + managedMemorySize.toString() + + ") exceed configured Total Flink Memory (" + totalFlinkMemorySize.toString() + ")."); + } + shuffleMemorySize = totalFlinkMemorySize.subtract(totalFlinkExcludeShuffleMemorySize); + sanityCheckShuffleMemory(config, shuffleMemorySize, totalFlinkMemorySize); + } else { + // derive shuffle memory from shuffle configs + if (isUsingLegacyShuffleConfigs(config)) { + shuffleMemorySize = getShuffleMemorySizeWithLegacyConfig(config); + } else { + shuffleMemorySize = deriveShuffleMemoryWithInverseFraction(config, totalFlinkExcludeShuffleMemorySize); + } + } + + final FlinkInternalMemory flinkInternalMemory = new FlinkInternalMemory( + frameworkHeapMemorySize, + taskHeapMemorySize, + taskOffHeapMemorySize, + shuffleMemorySize, + onHeapAndOffHeapManagedMemory.onHeap, + onHeapAndOffHeapManagedMemory.offHeap); + sanityCheckTotalFlinkMemory(config, flinkInternalMemory); + + // derive jvm metaspace and overhead + + final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = deriveJvmMetaspaceAndOverheadFromTotalFlinkMemory(config, flinkInternalMemory.getTotalFlinkMemorySize()); + sanityCheckTotalProcessMemory(config, flinkInternalMemory.getTotalFlinkMemorySize(), jvmMetaspaceAndOverhead); + + return createTaskExecutorResourceSpec(flinkInternalMemory, jvmMetaspaceAndOverhead); + } + + private static TaskExecutorResourceSpec deriveResourceSpecWithTotalFlinkMemory(final Configuration config) { + // derive flink internal memory from explicitly configured total flink memory + + final MemorySize totalFlinkMemorySize = getTotalFlinkMemorySize(config); + final FlinkInternalMemory flinkInternalMemory = deriveInternalMemoryFromTotalFlinkMemory(config, totalFlinkMemorySize); + + // derive jvm metaspace and overhead + + final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = deriveJvmMetaspaceAndOverheadFromTotalFlinkMemory(config, totalFlinkMemorySize); + sanityCheckTotalProcessMemory(config, totalFlinkMemorySize, jvmMetaspaceAndOverhead); + + return createTaskExecutorResourceSpec(flinkInternalMemory, jvmMetaspaceAndOverhead); + } + + private static TaskExecutorResourceSpec deriveResourceSpecWithTotalProcessMemory(final Configuration config) { + // derive total flink memory from explicitly configured total process memory size + + final MemorySize totalProcessMemorySize = getTotalProcessMemorySize(config); + final MemorySize jvmMetaspaceSize = getJvmMetaspaceSize(config); + final MemorySize jvmOverheadSize = deriveJvmOverheadWithFraction(config, totalProcessMemorySize); + final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = new JvmMetaspaceAndOverhead(jvmMetaspaceSize, jvmOverheadSize); + + if (jvmMetaspaceAndOverhead.getTotalJvmMetaspaceAndOverheadSize().getBytes() > totalProcessMemorySize.getBytes()) { + throw new IllegalConfigurationException( + "Sum of configured JVM Metaspace (" + jvmMetaspaceAndOverhead.metaspace.toString() + + ") and JVM Overhead (" + jvmMetaspaceAndOverhead.overhead.toString() + + ") exceed configured Total Process Memory (" + totalProcessMemorySize.toString() + ")."); + } + final MemorySize totalFlinkMemorySize = totalProcessMemorySize.subtract(jvmMetaspaceAndOverhead.getTotalJvmMetaspaceAndOverheadSize()); + + // derive flink internal memory + + final FlinkInternalMemory flinkInternalMemory = deriveInternalMemoryFromTotalFlinkMemory(config, totalFlinkMemorySize); + + return createTaskExecutorResourceSpec(flinkInternalMemory, jvmMetaspaceAndOverhead); + } + + private static JvmMetaspaceAndOverhead deriveJvmMetaspaceAndOverheadFromTotalFlinkMemory( + final Configuration config, final MemorySize totalFlinkMemorySize) { + final MemorySize jvmMetaspaceSize = getJvmMetaspaceSize(config); + final MemorySize jvmOverheadSize = deriveJvmOverheadWithInverseFraction(config, + totalFlinkMemorySize.add(jvmMetaspaceSize)); + return new JvmMetaspaceAndOverhead(jvmMetaspaceSize, jvmOverheadSize); + } + + private static FlinkInternalMemory deriveInternalMemoryFromTotalFlinkMemory( + final Configuration config, final MemorySize totalFlinkMemorySize) { + final MemorySize frameworkHeapMemorySize = getFrameworkHeapMemorySize(config); + final MemorySize taskOffHeapMemorySize = getTaskOffHeapMemorySize(config); + + final MemorySize taskHeapMemorySize; + final MemorySize shuffleMemorySize; + final MemorySize managedMemorySize; + + if (isTaskHeapMemorySizeExplicitlyConfigured(config)) { + // task heap memory is configured, + // derive managed memory first, leave the remaining to shuffle memory and check against shuffle min/max + taskHeapMemorySize = getTaskHeapMemorySize(config); + managedMemorySize = deriveManagedMemoryAbsoluteOrWithFraction(config, totalFlinkMemorySize); + final MemorySize totalFlinkExcludeShuffleMemorySize = + frameworkHeapMemorySize.add(taskHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize); + if (totalFlinkExcludeShuffleMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { + throw new IllegalConfigurationException( + "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toString() + + "), Task Heap Memory (" + taskHeapMemorySize.toString() + + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toString() + + ") and Managed Memory (" + managedMemorySize.toString() + + ") exceed configured Total Flink Memory (" + totalFlinkMemorySize.toString() + ")."); + } + shuffleMemorySize = totalFlinkMemorySize.subtract(totalFlinkExcludeShuffleMemorySize); + sanityCheckShuffleMemory(config, shuffleMemorySize, totalFlinkMemorySize); + } else { + // task heap memory is not configured + // derive managed memory and shuffle memory, leave the remaining to task heap memory + if (isManagedMemorySizeExplicitlyConfigured(config)) { + managedMemorySize = getManagedMemorySize(config); + } else { + managedMemorySize = deriveManagedMemoryAbsoluteOrWithFraction(config, totalFlinkMemorySize); + } + shuffleMemorySize = deriveShuffleMemoryWithFraction(config, totalFlinkMemorySize); + final MemorySize totalFlinkExcludeTaskHeapMemorySize = + frameworkHeapMemorySize.add(taskOffHeapMemorySize).add(managedMemorySize).add(shuffleMemorySize); + if (totalFlinkExcludeTaskHeapMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { + throw new IllegalConfigurationException( + "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toString() + + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toString() + + "), Managed Memory (" + managedMemorySize.toString() + + ") and Shuffle Memory (" + shuffleMemorySize.toString() + + ") exceed configured Total Flink Memory (" + totalFlinkMemorySize.toString() + ")."); + } + taskHeapMemorySize = totalFlinkMemorySize.subtract(totalFlinkExcludeTaskHeapMemorySize); + } + + final OnHeapAndOffHeapManagedMemory onHeapAndOffHeapManagedMemory = deriveOnHeapAndOffHeapMemoryFromManagedMemory(config, managedMemorySize); + final FlinkInternalMemory flinkInternalMemory = new FlinkInternalMemory( + frameworkHeapMemorySize, + taskHeapMemorySize, + taskOffHeapMemorySize, + shuffleMemorySize, + onHeapAndOffHeapManagedMemory.onHeap, + onHeapAndOffHeapManagedMemory.offHeap); + sanityCheckTotalFlinkMemory(config, flinkInternalMemory); + + return flinkInternalMemory; + } + + private static OnHeapAndOffHeapManagedMemory deriveOnHeapAndOffHeapMemoryFromManagedMemory( + final Configuration config, final MemorySize managedMemorySize) { + + final MemorySize offHeapSize; + + if (isManagedMemoryOffHeapSizeExplicitlyConfigured(config)) { + offHeapSize = getManagedMemoryOffHeapSize(config); + // sanity check + if (offHeapSize.getBytes() > managedMemorySize.getBytes()) { + throw new IllegalConfigurationException("Configured Off-Heap Managed Memory size (" + offHeapSize.toString() + + ") is larger than configured/derived total Managed Memory size (" + managedMemorySize.toString() + ")."); + } + } else { + final double offHeapFraction; + if (isManagedMemoryOffHeapFractionExplicitlyConfigured(config)) { + offHeapFraction = getManagedMemoryOffHeapFraction(config); + } else { + @SuppressWarnings("deprecation") + final boolean legacyManagedMemoryOffHeap = config.getBoolean(TaskManagerOptions.MEMORY_OFF_HEAP); + offHeapFraction = legacyManagedMemoryOffHeap ? 1.0 : 0.0; + } + offHeapSize = managedMemorySize.multiply(offHeapFraction); + } + + final MemorySize onHeapSize = managedMemorySize.subtract(offHeapSize); + return new OnHeapAndOffHeapManagedMemory(onHeapSize, offHeapSize); + } + + private static MemorySize deriveManagedMemoryAbsoluteOrWithFraction(final Configuration config, final MemorySize base) { + if (isManagedMemorySizeExplicitlyConfigured(config)) { + return getManagedMemorySize(config); + } else { + return deriveWithFraction(base, getManagedMemoryRangeFraction(config)); + } + } + + private static MemorySize deriveShuffleMemoryWithFraction(final Configuration config, final MemorySize base) { + return deriveWithFraction(base, getShuffleMemoryRangeFraction(config)); + } + + private static MemorySize deriveShuffleMemoryWithInverseFraction(final Configuration config, final MemorySize base) { + return deriveWithInverseFraction(base, getShuffleMemoryRangeFraction(config)); + } + + private static MemorySize deriveJvmOverheadWithFraction(final Configuration config, final MemorySize base) { + return deriveWithFraction(base, getJvmOverheadRangeFraction(config)); + } + + private static MemorySize deriveJvmOverheadWithInverseFraction(final Configuration config, final MemorySize base) { + return deriveWithInverseFraction(base, getJvmOverheadRangeFraction(config)); + } + + private static MemorySize deriveWithFraction(final MemorySize base, final RangeFraction rangeFraction) { + final long relative = (long) (rangeFraction.fraction * base.getBytes()); + return new MemorySize(Math.max(rangeFraction.minSize.getBytes(), Math.min(rangeFraction.maxSize.getBytes(), relative))); + } + + private static MemorySize deriveWithInverseFraction(final MemorySize base, final RangeFraction rangeFraction) { + checkArgument(rangeFraction.fraction < 1); + final long relative = (long) (rangeFraction.fraction / (1 - rangeFraction.fraction) * base.getBytes()); + return new MemorySize(Math.max(rangeFraction.minSize.getBytes(), Math.min(rangeFraction.maxSize.getBytes(), relative))); + } + + private static MemorySize getFrameworkHeapMemorySize(final Configuration config) { + return MemorySize.parse(config.getString(TaskManagerOptions.FRAMEWORK_HEAP_MEMORY)); + } + + private static MemorySize getTaskHeapMemorySize(final Configuration config) { + checkArgument(isTaskHeapMemorySizeExplicitlyConfigured(config)); + return MemorySize.parse(config.getString(TaskManagerOptions.TASK_HEAP_MEMORY)); + } + + private static MemorySize getTaskOffHeapMemorySize(final Configuration config) { + return MemorySize.parse(config.getString(TaskManagerOptions.TASK_OFF_HEAP_MEMORY)); + } + + private static MemorySize getManagedMemorySize(final Configuration config) { + checkArgument(isManagedMemorySizeExplicitlyConfigured(config)); + return MemorySize.parse(config.getString(TaskManagerOptions.MANAGED_MEMORY_SIZE)); + } + + private static RangeFraction getManagedMemoryRangeFraction(final Configuration config) { + final MemorySize minSize = new MemorySize(0); + final MemorySize maxSize = new MemorySize(Long.MAX_VALUE); + final double fraction = config.getFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION); + if (fraction >= 1 || fraction < 0) { + throw new IllegalConfigurationException("Configured Managed Memory fraction (" + + fraction + ") must be in [0, 1)."); + } + return new RangeFraction(minSize, maxSize, fraction); + } + + private static double getManagedMemoryOffHeapFraction(final Configuration config) { + checkArgument(isManagedMemoryOffHeapFractionExplicitlyConfigured(config)); + final double offHeapFraction = config.getFloat(TaskManagerOptions.MANAGED_MEMORY_OFFHEAP_FRACTION); + if (offHeapFraction > 1 || offHeapFraction < 0) { + throw new IllegalConfigurationException("Configured Off-Heap Managed Memory fraction (" + + offHeapFraction + ") must be in [0, 1]."); + } + return offHeapFraction; + } + + private static MemorySize getManagedMemoryOffHeapSize(final Configuration config) { + checkArgument(isManagedMemoryOffHeapSizeExplicitlyConfigured(config)); + return MemorySize.parse(config.getString(TaskManagerOptions.MANAGED_MEMORY_OFFHEAP_SIZE)); + } + + private static MemorySize getShuffleMemorySizeWithLegacyConfig(final Configuration config) { + checkArgument(isUsingLegacyShuffleConfigs(config)); + @SuppressWarnings("deprecation") + final long numOfBuffers = config.getInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS); + final long pageSize = ConfigurationParserUtils.getPageSize(config); + return new MemorySize(numOfBuffers * pageSize); + } + + private static RangeFraction getShuffleMemoryRangeFraction(final Configuration config) { + final MemorySize minSize = MemorySize.parse(config.getString(TaskManagerOptions.SHUFFLE_MEMORY_MIN)); + final MemorySize maxSize = MemorySize.parse(config.getString(TaskManagerOptions.SHUFFLE_MEMORY_MAX)); + final double fraction = config.getFloat(TaskManagerOptions.SHUFFLE_MEMORY_FRACTION); + if (fraction >= 1 || fraction < 0) { + throw new IllegalConfigurationException("Configured Shuffle Memory fraction (" + + fraction + ") must be in [0, 1)."); + } + return new RangeFraction(minSize, maxSize, fraction); + } + + private static MemorySize getJvmMetaspaceSize(final Configuration config) { + return MemorySize.parse(config.getString(TaskManagerOptions.JVM_METASPACE)); + } + + private static RangeFraction getJvmOverheadRangeFraction(final Configuration config) { + final MemorySize minSize = MemorySize.parse(config.getString(TaskManagerOptions.JVM_OVERHEAD_MIN)); + final MemorySize maxSize = MemorySize.parse(config.getString(TaskManagerOptions.JVM_OVERHEAD_MAX)); + final double fraction = config.getFloat(TaskManagerOptions.JVM_OVERHEAD_FRACTION); + if (fraction >= 1 || fraction < 0) { + throw new IllegalConfigurationException("Configured JVM Overhead fraction (" + + fraction + ") must be in [0, 1)."); + } + return new RangeFraction(minSize, maxSize, fraction); + } + + private static MemorySize getTotalFlinkMemorySize(final Configuration config) { + checkArgument(isTotalFlinkMemorySizeExplicitlyConfigured(config)); + if (config.contains(TaskManagerOptions.TOTAL_FLINK_MEMORY)) { + return MemorySize.parse(config.getString(TaskManagerOptions.TOTAL_FLINK_MEMORY)); + } else { + @SuppressWarnings("deprecation") + final long legacyHeapMemoryMB = config.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB); + return new MemorySize(legacyHeapMemoryMB << 20); // megabytes to bytes + } + } + + private static MemorySize getTotalProcessMemorySize(final Configuration config) { + checkArgument(isTotalProcessMemorySizeExplicitlyConfigured(config)); + return MemorySize.parse(config.getString(TaskManagerOptions.TOTAL_PROCESS_MEMORY)); + } + + private static boolean isTaskHeapMemorySizeExplicitlyConfigured(final Configuration config) { + return config.contains(TaskManagerOptions.TASK_HEAP_MEMORY); + } + + private static boolean isManagedMemorySizeExplicitlyConfigured(final Configuration config) { + return config.contains(TaskManagerOptions.MANAGED_MEMORY_SIZE); + } + + private static boolean isManagedMemoryOffHeapFractionExplicitlyConfigured(final Configuration config) { + return config.getFloat(TaskManagerOptions.MANAGED_MEMORY_OFFHEAP_FRACTION) >= 0; + } + + private static boolean isManagedMemoryOffHeapSizeExplicitlyConfigured(final Configuration config) { + return config.contains(TaskManagerOptions.MANAGED_MEMORY_OFFHEAP_SIZE); + } + + private static boolean isUsingLegacyShuffleConfigs(final Configuration config) { + // use the legacy number-of-buffer config option only when it is explicitly configured and + // none of new config options is explicitly configured + @SuppressWarnings("deprecation") + final boolean legacyConfigured = config.contains(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS); + return !config.contains(TaskManagerOptions.SHUFFLE_MEMORY_MIN) && + !config.contains(TaskManagerOptions.SHUFFLE_MEMORY_MAX) && + !config.contains(TaskManagerOptions.SHUFFLE_MEMORY_FRACTION) && + legacyConfigured; + } + + private static boolean isShuffleMemoryFractionExplicitlyConfigured(final Configuration config) { + return config.contains(TaskManagerOptions.SHUFFLE_MEMORY_FRACTION); + } + + private static boolean isTotalFlinkMemorySizeExplicitlyConfigured(final Configuration config) { + // backward compatible with the deprecated config option TASK_MANAGER_HEAP_MEMORY_MB only when it's explicitly + // configured by the user + @SuppressWarnings("deprecation") + final boolean legacyConfigured = config.contains(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB); + return config.contains(TaskManagerOptions.TOTAL_FLINK_MEMORY) || legacyConfigured; + } + + private static boolean isTotalProcessMemorySizeExplicitlyConfigured(final Configuration config) { + return config.contains(TaskManagerOptions.TOTAL_PROCESS_MEMORY); + } + + private static void sanityCheckTotalFlinkMemory(final Configuration config, final FlinkInternalMemory flinkInternalMemory) { + if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) { + final MemorySize configuredTotalFlinkMemorySize = getTotalFlinkMemorySize(config); + if (!configuredTotalFlinkMemorySize.equals(flinkInternalMemory.getTotalFlinkMemorySize())) { + throw new IllegalConfigurationException( + "Configured/Derived Flink internal memory sizes (total " + flinkInternalMemory.getTotalFlinkMemorySize().toString() + + ") do not add up to the configured Total Flink Memory size (" + configuredTotalFlinkMemorySize.toString() + + "). Configured/Derived Flink internal memory sizes are: " + + "Framework Heap Memory (" + flinkInternalMemory.frameworkHeap.toString() + + "), Task Heap Memory (" + flinkInternalMemory.taskHeap.toString() + + "), Task Off-Heap Memory (" + flinkInternalMemory.taskOffHeap.toString() + + "), Shuffle Memory (" + flinkInternalMemory.shuffle.toString() + + "), Managed Memory (" + flinkInternalMemory.getManagedMemorySize().toString() + ")."); + } + } + } + + private static void sanityCheckTotalProcessMemory(final Configuration config, final MemorySize totalFlinkMemory, final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead) { + final MemorySize derivedTotalProcessMemorySize = + totalFlinkMemory.add(jvmMetaspaceAndOverhead.metaspace).add(jvmMetaspaceAndOverhead.overhead); + if (isTotalProcessMemorySizeExplicitlyConfigured(config)) { + final MemorySize configuredTotalProcessMemorySize = getTotalProcessMemorySize(config); + if (!configuredTotalProcessMemorySize.equals(derivedTotalProcessMemorySize)) { + throw new IllegalConfigurationException( + "Configured/Derived memory sizes (total " + derivedTotalProcessMemorySize.toString() + + ") do not add up to the configured Total Process Memory size (" + configuredTotalProcessMemorySize.toString() + + "). Configured/Derived memory sizes are: " + + "Total Flink Memory (" + totalFlinkMemory.toString() + + "), JVM Metaspace (" + jvmMetaspaceAndOverhead.metaspace.toString() + + "), JVM Overhead (" + jvmMetaspaceAndOverhead.overhead.toString() + ")."); + } + } + } + + private static void sanityCheckShuffleMemory(final Configuration config, final MemorySize derivedShuffleMemorySize, final MemorySize totalFlinkMemorySize) { + if (isUsingLegacyShuffleConfigs(config)) { + final MemorySize configuredShuffleMemorySize = getShuffleMemorySizeWithLegacyConfig(config); + if (!configuredShuffleMemorySize.equals(derivedShuffleMemorySize)) { + throw new IllegalConfigurationException( + "Derived Shuffle Memory size (" + derivedShuffleMemorySize.toString() + + ") does not match configured Shuffle Memory size (" + configuredShuffleMemorySize.toString() + ")."); + } + } else { + final RangeFraction shuffleRangeFraction = getShuffleMemoryRangeFraction(config); + if (derivedShuffleMemorySize.getBytes() > shuffleRangeFraction.maxSize.getBytes() || + derivedShuffleMemorySize.getBytes() < shuffleRangeFraction.minSize.getBytes()) { + throw new IllegalConfigurationException("Derived Shuffle Memory size (" + + derivedShuffleMemorySize.toString() + ") is not in configured Shuffle Memory range [" + + shuffleRangeFraction.minSize.toString() + ", " + + shuffleRangeFraction.maxSize.toString() + "]."); + } + if (isShuffleMemoryFractionExplicitlyConfigured(config) && + !derivedShuffleMemorySize.equals(totalFlinkMemorySize.multiply(shuffleRangeFraction.fraction))) { + throw new IllegalConfigurationException("Derived Shuffle Memory size(" + + derivedShuffleMemorySize.toString() + ") does not match configured Shuffle Memory fraction (" + + shuffleRangeFraction.fraction + ")."); + } + } + } + + private static TaskExecutorResourceSpec createTaskExecutorResourceSpec( + final FlinkInternalMemory flinkInternalMemory, final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead) { + return new TaskExecutorResourceSpec( + flinkInternalMemory.frameworkHeap, + flinkInternalMemory.taskHeap, + flinkInternalMemory.taskOffHeap, + flinkInternalMemory.shuffle, + flinkInternalMemory.onHeapManaged, + flinkInternalMemory.offHeapManaged, + jvmMetaspaceAndOverhead.metaspace, + jvmMetaspaceAndOverhead.overhead); + } + + private static class RangeFraction { + final MemorySize minSize; + final MemorySize maxSize; + final double fraction; + + RangeFraction(final MemorySize minSize, final MemorySize maxSize, final double fraction) { + this.minSize = minSize; + this.maxSize = maxSize; + this.fraction = fraction; + checkArgument(minSize.getBytes() <= maxSize.getBytes()); + checkArgument(fraction >= 0 && fraction <= 1); + } + } + + private static class FlinkInternalMemory { + final MemorySize frameworkHeap; + final MemorySize taskHeap; + final MemorySize taskOffHeap; + final MemorySize shuffle; + final MemorySize onHeapManaged; + final MemorySize offHeapManaged; + + FlinkInternalMemory( + final MemorySize frameworkHeap, + final MemorySize taskHeap, + final MemorySize taskOffHeap, + final MemorySize shuffle, + final MemorySize onHeapManaged, + final MemorySize offHeapManaged) { + + this.frameworkHeap = checkNotNull(frameworkHeap); + this.taskHeap = checkNotNull(taskHeap); + this.taskOffHeap = checkNotNull(taskOffHeap); + this.shuffle = checkNotNull(shuffle); + this.onHeapManaged = checkNotNull(onHeapManaged); + this.offHeapManaged = checkNotNull(offHeapManaged); + } + + MemorySize getTotalFlinkMemorySize() { + return frameworkHeap.add(taskHeap).add(taskOffHeap).add(shuffle).add(getManagedMemorySize()); + } + + MemorySize getManagedMemorySize() { + return onHeapManaged.add(offHeapManaged); + } + } + + private static class OnHeapAndOffHeapManagedMemory { + final MemorySize onHeap; + final MemorySize offHeap; + + OnHeapAndOffHeapManagedMemory(final MemorySize onHeap, final MemorySize offHeap) { + this.onHeap = onHeap; + this.offHeap = offHeap; + } + } + + private static class JvmMetaspaceAndOverhead { + final MemorySize metaspace; + final MemorySize overhead; + + JvmMetaspaceAndOverhead(final MemorySize jvmMetaspace, final MemorySize jvmOverhead) { + this.metaspace = checkNotNull(jvmMetaspace); + this.overhead = checkNotNull(jvmOverhead); + } + + MemorySize getTotalJvmMetaspaceAndOverheadSize() { + return metaspace.add(overhead); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtilsTest.java new file mode 100644 index 0000000000..a77370d38e --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtilsTest.java @@ -0,0 +1,655 @@ +/* + * 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.clusterframework; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.function.Consumer; + +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * Tests for {@link TaskExecutorResourceUtils}. + */ +public class TaskExecutorResourceUtilsTest extends TestLogger { + + private static final MemorySize TASK_HEAP_SIZE = MemorySize.parse("100m"); + private static final MemorySize MANAGED_MEM_SIZE = MemorySize.parse("200m"); + private static final MemorySize TOTAL_FLINK_MEM_SIZE = MemorySize.parse("800m"); + private static final MemorySize TOTAL_PROCESS_MEM_SIZE = MemorySize.parse("1g"); + + @Test + public void testConfigFrameworkHeapMemory() { + final MemorySize frameworkHeapSize = MemorySize.parse("100m"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.FRAMEWORK_HEAP_MEMORY, frameworkHeapSize.getMebiBytes() + "m"); + + validateInAllConfigurations(conf, taskExecutorResourceSpec -> assertThat(taskExecutorResourceSpec.getFrameworkHeapSize(), is(frameworkHeapSize))); + } + + @Test + public void testConfigTaskHeapMemory() { + final MemorySize taskHeapSize = MemorySize.parse("50m"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.TASK_HEAP_MEMORY, taskHeapSize.getMebiBytes() + "m"); + + // validate in configurations without explicit task heap memory size, + // to avoid checking against overwritten task heap memory size + validateInConfigurationsWithoutExplicitTaskHeapMem(conf, taskExecutorResourceSpec -> assertThat(taskExecutorResourceSpec.getTaskHeapSize(), is(taskHeapSize))); + } + + @Test + public void testConfigTaskOffheapMemory() { + final MemorySize taskOffHeapSize = MemorySize.parse("50m"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.TASK_OFF_HEAP_MEMORY, taskOffHeapSize.getMebiBytes() + "m"); + + validateInAllConfigurations(conf, taskExecutorResourceSpec -> assertThat(taskExecutorResourceSpec.getTaskOffHeapSize(), is(taskOffHeapSize))); + } + + @Test + public void testConfigShuffleMemoryRange() { + final MemorySize shuffleMin = MemorySize.parse("50m"); + final MemorySize shuffleMax = MemorySize.parse("200m"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.SHUFFLE_MEMORY_MAX, shuffleMax.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.SHUFFLE_MEMORY_MIN, shuffleMin.getMebiBytes() + "m"); + + validateInAllConfigurations(conf, taskExecutorResourceSpec -> { + assertThat(taskExecutorResourceSpec.getShuffleMemSize().getBytes(), greaterThanOrEqualTo(shuffleMin.getBytes())); + assertThat(taskExecutorResourceSpec.getShuffleMemSize().getBytes(), lessThanOrEqualTo(shuffleMax.getBytes())); + }); + } + + @Test + public void testConfigShuffleMemoryRangeFailure() { + final MemorySize shuffleMin = MemorySize.parse("200m"); + final MemorySize shuffleMax = MemorySize.parse("50m"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.SHUFFLE_MEMORY_MAX, shuffleMax.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.SHUFFLE_MEMORY_MIN, shuffleMin.getMebiBytes() + "m"); + + validateFailInAllConfigurations(conf); + } + + @Test + public void testConfigShuffleMemoryFraction() { + final MemorySize shuffleMin = MemorySize.parse("0m"); + final MemorySize shuffleMax = MemorySize.parse("1t"); + final float fraction = 0.2f; + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.SHUFFLE_MEMORY_MAX, shuffleMax.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.SHUFFLE_MEMORY_MIN, shuffleMin.getMebiBytes() + "m"); + conf.setFloat(TaskManagerOptions.SHUFFLE_MEMORY_FRACTION, fraction); + + // validate in configurations without explicit total flink/process memory, otherwise explicit configured + // shuffle memory fraction might conflict with total flink/process memory minus other memory sizes + validateInConfigWithExplicitTaskHeapAndManagedMem(conf, taskExecutorResourceSpec -> + assertThat(taskExecutorResourceSpec.getShuffleMemSize(), is(taskExecutorResourceSpec.getTotalFlinkMemorySize().multiply(fraction)))); + } + + @Test + public void testConfigShuffleMemoryFractionFailure() { + Configuration conf = new Configuration(); + conf.setFloat(TaskManagerOptions.SHUFFLE_MEMORY_FRACTION, -0.1f); + validateFailInAllConfigurations(conf); + + conf.setFloat(TaskManagerOptions.SHUFFLE_MEMORY_FRACTION, 1.0f); + validateFailInAllConfigurations(conf); + } + + @Test + public void testConfigShuffleMemoryLegacyRangeFraction() { + final MemorySize shuffleMin = MemorySize.parse("50m"); + final MemorySize shuffleMax = MemorySize.parse("200m"); + final float fraction = 0.2f; + + @SuppressWarnings("deprecation") + final ConfigOption legacyOptionMin = NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN; + @SuppressWarnings("deprecation") + final ConfigOption legacyOptionMax = NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX; + @SuppressWarnings("deprecation") + final ConfigOption legacyOptionFraction = NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION; + + Configuration conf = new Configuration(); + conf.setString(legacyOptionMin, shuffleMin.getMebiBytes() + "m"); + conf.setString(legacyOptionMax, shuffleMax.getMebiBytes() + "m"); + + validateInAllConfigurations(conf, taskExecutorResourceSpec -> { + assertThat(taskExecutorResourceSpec.getShuffleMemSize().getBytes(), greaterThanOrEqualTo(shuffleMin.getBytes())); + assertThat(taskExecutorResourceSpec.getShuffleMemSize().getBytes(), lessThanOrEqualTo(shuffleMax.getBytes())); + }); + + conf.setString(legacyOptionMin, "0m"); + conf.setString(legacyOptionMax, "1t"); + conf.setFloat(legacyOptionFraction, fraction); + + validateInConfigWithExplicitTaskHeapAndManagedMem(conf, taskExecutorResourceSpec -> + assertThat(taskExecutorResourceSpec.getShuffleMemSize(), is(taskExecutorResourceSpec.getTotalFlinkMemorySize().multiply(fraction)))); + } + + @Test + public void testConfigShuffleMemoryLegacyNumOfBuffers() { + final MemorySize pageSize = MemorySize.parse("32k"); + final int numOfBuffers = 1024; + final MemorySize shuffleSize = pageSize.multiply(numOfBuffers); + + @SuppressWarnings("deprecation") + final ConfigOption legacyOption = NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS; + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.MEMORY_SEGMENT_SIZE, pageSize.getKibiBytes() + "k"); + conf.setInteger(legacyOption, numOfBuffers); + + // validate in configurations without explicit total flink/process memory, otherwise explicit configured + // shuffle memory size might conflict with total flink/process memory minus other memory sizes + validateInConfigWithExplicitTaskHeapAndManagedMem(conf, taskExecutorResourceSpec -> + assertThat(taskExecutorResourceSpec.getShuffleMemSize(), is(shuffleSize))); + } + + @Test + public void testConfigManagedMemorySize() { + final MemorySize managedMemSize = MemorySize.parse("100m"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, managedMemSize.getMebiBytes() + "m"); + + // validate in configurations without explicit managed memory size, + // to avoid checking against overwritten managed memory size + validateInConfigurationsWithoutExplicitManagedMem(conf, taskExecutorResourceSpec -> assertThat(taskExecutorResourceSpec.getManagedMemorySize(), is(managedMemSize))); + } + + @Test + public void testConfigManagedMemoryLegacySize() { + final MemorySize managedMemSize = MemorySize.parse("100m"); + + @SuppressWarnings("deprecation") + final ConfigOption legacyOption = TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE; + + Configuration conf = new Configuration(); + conf.setString(legacyOption, managedMemSize.getMebiBytes() + "m"); + + // validate in configurations without explicit managed memory size, + // to avoid checking against overwritten managed memory size + validateInConfigurationsWithoutExplicitManagedMem(conf, taskExecutorResourceSpec -> assertThat(taskExecutorResourceSpec.getManagedMemorySize(), is(managedMemSize))); + } + + @Test + public void testConfigManagedMemoryFraction() { + final float fraction = 0.5f; + + Configuration conf = new Configuration(); + conf.setFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION, fraction); + + // managed memory fraction is only used when managed memory size is not explicitly configured + validateInConfigurationsWithoutExplicitManagedMem(conf, taskExecutorResourceSpec -> + assertThat(taskExecutorResourceSpec.getManagedMemorySize(), is(taskExecutorResourceSpec.getTotalFlinkMemorySize().multiply(fraction)))); + } + + @Test + public void testConfigManagedMemoryFractionFailure() { + final Configuration conf = new Configuration(); + conf.setFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION, -0.1f); + validateFailInConfigurationsWithoutExplicitManagedMem(conf); + + conf.setFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION, 1.0f); + validateFailInConfigurationsWithoutExplicitManagedMem(conf); + } + + @Test + public void testConfigManagedMemoryLegacyFraction() { + final float fraction = 0.5f; + + @SuppressWarnings("deprecation") + final ConfigOption legacyOption = TaskManagerOptions.LEGACY_MANAGED_MEMORY_FRACTION; + + Configuration conf = new Configuration(); + conf.setFloat(legacyOption, fraction); + + // managed memory fraction is only used when managed memory size is not explicitly configured + validateInConfigurationsWithoutExplicitManagedMem(conf, taskExecutorResourceSpec -> + assertThat(taskExecutorResourceSpec.getManagedMemorySize(), is(taskExecutorResourceSpec.getTotalFlinkMemorySize().multiply(fraction)))); + } + + @Test + public void testConfigOffHeapManagedMemorySize() { + final MemorySize offHeapSize = MemorySize.parse("20m"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.MANAGED_MEMORY_OFFHEAP_SIZE, offHeapSize.getMebiBytes() + "m"); + + validateInAllConfigurations(conf, taskExecutorResourceSpec -> { + assertThat(taskExecutorResourceSpec.getOffHeapManagedMemorySize(), is(offHeapSize)); + assertThat(taskExecutorResourceSpec.getOnHeapManagedMemorySize(), is(taskExecutorResourceSpec.getManagedMemorySize().subtract(taskExecutorResourceSpec.getOffHeapManagedMemorySize()))); + }); + } + + @Test + public void testConfigOffHeapManagedMemorySizeFailure() { + final MemorySize offHeapSize = MemorySize.parse("1t"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.MANAGED_MEMORY_OFFHEAP_SIZE, offHeapSize.getMebiBytes() + "m"); + + validateFailInAllConfigurations(conf); + } + + @Test + public void testConfigOffHeapManagedMemoryFraction() { + final float fraction = 0.5f; + + Configuration conf = new Configuration(); + conf.setFloat(TaskManagerOptions.MANAGED_MEMORY_OFFHEAP_FRACTION, fraction); + + validateInAllConfigurations(conf, taskExecutorResourceSpec -> { + assertThat(taskExecutorResourceSpec.getOffHeapManagedMemorySize(), is(taskExecutorResourceSpec.getManagedMemorySize().multiply(fraction))); + assertThat(taskExecutorResourceSpec.getOnHeapManagedMemorySize(), is(taskExecutorResourceSpec.getManagedMemorySize().subtract(taskExecutorResourceSpec.getOffHeapManagedMemorySize()))); + }); + } + + @Test + public void testConfigOffHeapManagedMemoryFractionFailure() { + Configuration conf = new Configuration(); + conf.setFloat(TaskManagerOptions.MANAGED_MEMORY_OFFHEAP_FRACTION, 1.1f); + validateFailInAllConfigurations(conf); + } + + @Test + public void testConfigOffHeapManagedMemoryLegacyOffHeap() { + @SuppressWarnings("deprecation") + final ConfigOption legacyOption = TaskManagerOptions.MEMORY_OFF_HEAP; + + // negative off-heap managed memory fraction means not configured, if off-heap managed memory size is also not configured, + // legacy 'taskmanager.memory.off-heap' will be used to set managed memory to either all on-heap or all off-heap + Configuration conf = new Configuration(); + conf.setFloat(TaskManagerOptions.MANAGED_MEMORY_OFFHEAP_FRACTION, -1.0f); + + conf.setBoolean(legacyOption, true); + validateInAllConfigurations(conf, taskExecutorResourceSpec -> { + assertThat(taskExecutorResourceSpec.getOffHeapManagedMemorySize(), is(taskExecutorResourceSpec.getManagedMemorySize())); + assertThat(taskExecutorResourceSpec.getOnHeapManagedMemorySize(), is(new MemorySize(0L))); + }); + } + + @Test + public void testConfigOffHeapManagedMemoryLegacyOnHeap() { + @SuppressWarnings("deprecation") + final ConfigOption legacyOption = TaskManagerOptions.MEMORY_OFF_HEAP; + + // negative off-heap managed memory fraction means not configured, if off-heap managed memory size is also not configured, + // legacy 'taskmanager.memory.off-heap' will be used to set managed memory to either all on-heap or all off-heap + Configuration conf = new Configuration(); + conf.setFloat(TaskManagerOptions.MANAGED_MEMORY_OFFHEAP_FRACTION, -1.0f); + + conf.setBoolean(legacyOption, false); + validateInAllConfigurations(conf, taskExecutorResourceSpec -> { + assertThat(taskExecutorResourceSpec.getOnHeapManagedMemorySize(), is(taskExecutorResourceSpec.getManagedMemorySize())); + assertThat(taskExecutorResourceSpec.getOffHeapManagedMemorySize(), is(new MemorySize(0L))); + }); + } + + @Test + public void testConfigJvmMetaspaceSize() { + final MemorySize jvmMetaspaceSize = MemorySize.parse("50m"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.JVM_METASPACE, jvmMetaspaceSize.getMebiBytes() + "m"); + + validateInAllConfigurations(conf, taskExecutorResourceSpec -> assertThat(taskExecutorResourceSpec.getJvmMetaspaceSize(), is(jvmMetaspaceSize))); + } + + @Test + public void testConfigJvmOverheadRange() { + final MemorySize minSize = MemorySize.parse("50m"); + final MemorySize maxSize = MemorySize.parse("200m"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.JVM_OVERHEAD_MAX, maxSize.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.JVM_OVERHEAD_MIN, minSize.getMebiBytes() + "m"); + + validateInAllConfigurations(conf, taskExecutorResourceSpec -> { + assertThat(taskExecutorResourceSpec.getJvmOverheadSize().getBytes(), + greaterThanOrEqualTo(minSize.getBytes())); + assertThat(taskExecutorResourceSpec.getJvmOverheadSize().getBytes(), lessThanOrEqualTo(maxSize.getBytes())); + }); + } + + @Test + public void testConfigJvmOverheadRangeFailure() { + final MemorySize minSize = MemorySize.parse("200m"); + final MemorySize maxSize = MemorySize.parse("50m"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.JVM_OVERHEAD_MAX, maxSize.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.JVM_OVERHEAD_MIN, minSize.getMebiBytes() + "m"); + + validateFailInAllConfigurations(conf); + } + + @Test + public void testConfigJvmOverheadFraction() { + final MemorySize minSize = MemorySize.parse("0m"); + final MemorySize maxSize = MemorySize.parse("1t"); + final float fraction = 0.2f; + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.JVM_OVERHEAD_MAX, maxSize.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.JVM_OVERHEAD_MIN, minSize.getMebiBytes() + "m"); + conf.setFloat(TaskManagerOptions.JVM_OVERHEAD_FRACTION, fraction); + + validateInAllConfigurations(conf, taskExecutorResourceSpec -> + assertThat(taskExecutorResourceSpec.getJvmOverheadSize(), is(taskExecutorResourceSpec.getTotalProcessMemorySize().multiply(fraction)))); + } + + @Test + public void testConfigJvmOverheadFractionFailureNegative() { + final Configuration conf = new Configuration(); + conf.setFloat(TaskManagerOptions.JVM_OVERHEAD_FRACTION, -0.1f); + validateFailInConfigurationsWithoutExplicitManagedMem(conf); + } + + @Test + public void testConfigJvmOverheadFractionFailureNoLessThanOne() { + final Configuration conf = new Configuration(); + conf.setFloat(TaskManagerOptions.JVM_OVERHEAD_FRACTION, 1.0f); + validateFailInConfigurationsWithoutExplicitManagedMem(conf); + } + + @Test + public void testConfigTotalFlinkMemory() { + final MemorySize totalFlinkMemorySize = MemorySize.parse("1g"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.TOTAL_FLINK_MEMORY, totalFlinkMemorySize.getMebiBytes() + "m"); + + TaskExecutorResourceSpec taskExecutorResourceSpec = TaskExecutorResourceUtils.resourceSpecFromConfig(conf); + assertThat(taskExecutorResourceSpec.getTotalFlinkMemorySize(), is(totalFlinkMemorySize)); + } + + @Test + public void testFlinkInternalMemorySizeAddUpFailure() { + final MemorySize totalFlinkMemory = MemorySize.parse("499m"); + final MemorySize frameworkHeap = MemorySize.parse("100m"); + final MemorySize taskHeap = MemorySize.parse("100m"); + final MemorySize taskOffHeap = MemorySize.parse("100m"); + final MemorySize shuffle = MemorySize.parse("100m"); + final MemorySize managed = MemorySize.parse("100m"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.TOTAL_FLINK_MEMORY, totalFlinkMemory.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.FRAMEWORK_HEAP_MEMORY, frameworkHeap.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.TASK_HEAP_MEMORY, taskHeap.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.TASK_OFF_HEAP_MEMORY, taskOffHeap.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.SHUFFLE_MEMORY_MIN, shuffle.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.SHUFFLE_MEMORY_MAX, shuffle.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, managed.getMebiBytes() + "m"); + + validateFail(conf); + } + + @Test + public void testConfigTotalFlinkMemoryLegacyMB() { + final MemorySize totalFlinkMemorySize = MemorySize.parse("1g"); + + @SuppressWarnings("deprecation") + final ConfigOption legacyOption = TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB; + + Configuration conf = new Configuration(); + conf.setInteger(legacyOption, totalFlinkMemorySize.getMebiBytes()); + + TaskExecutorResourceSpec taskExecutorResourceSpec = TaskExecutorResourceUtils.resourceSpecFromConfig(conf); + assertThat(taskExecutorResourceSpec.getTotalFlinkMemorySize(), is(totalFlinkMemorySize)); + } + + @Test + public void testConfigTotalProcessMemorySize() { + final MemorySize totalProcessMemorySize = MemorySize.parse("1g"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.TOTAL_PROCESS_MEMORY, totalProcessMemorySize.getMebiBytes() + "m"); + + TaskExecutorResourceSpec taskExecutorResourceSpec = TaskExecutorResourceUtils.resourceSpecFromConfig(conf); + assertThat(taskExecutorResourceSpec.getTotalProcessMemorySize(), is(totalProcessMemorySize)); + } + + @Test + public void testFlinkInternalMemoryFractionAddUpFailure() { + final float shuffleFraction = 0.6f; + final float managedFraction = 0.6f; + + Configuration conf = new Configuration(); + conf.setFloat(TaskManagerOptions.SHUFFLE_MEMORY_FRACTION, shuffleFraction); + conf.setFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION, managedFraction); + + // if managed memory size is explicitly configured, then managed memory fraction will be ignored + validateFailInConfigurationsWithoutExplicitManagedMem(conf); + } + + @Test + public void testConfigTotalFlinkMemoryLegacySize() { + final MemorySize totalFlinkMemorySize = MemorySize.parse("1g"); + + @SuppressWarnings("deprecation") + final ConfigOption legacyOption = TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY; + + Configuration conf = new Configuration(); + conf.setString(legacyOption, totalFlinkMemorySize.getMebiBytes() + "m"); + + TaskExecutorResourceSpec taskExecutorResourceSpec = TaskExecutorResourceUtils.resourceSpecFromConfig(conf); + assertThat(taskExecutorResourceSpec.getTotalFlinkMemorySize(), is(totalFlinkMemorySize)); + } + + @Test + public void testConfigTotalProcessMemoryAddUpFailure() { + final MemorySize totalProcessMemory = MemorySize.parse("699m"); + final MemorySize totalFlinkMemory = MemorySize.parse("500m"); + final MemorySize jvmMetaspace = MemorySize.parse("100m"); + final MemorySize jvmOverhead = MemorySize.parse("100m"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.TOTAL_PROCESS_MEMORY, totalProcessMemory.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.TOTAL_FLINK_MEMORY, totalFlinkMemory.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.JVM_METASPACE, jvmMetaspace.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.JVM_OVERHEAD_MIN, jvmOverhead.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.JVM_OVERHEAD_MAX, jvmOverhead.getMebiBytes() + "m"); + + validateFail(conf); + } + + private void validateInAllConfigurations(final Configuration customConfig, Consumer validateFunc) { + validateInConfigWithExplicitTaskHeapAndManagedMem(customConfig, validateFunc); + validateInConfigWithExplicitTotalFlinkMem(customConfig, validateFunc); + validateInConfigWithExplicitTotalFlinkAndTaskHeapMem(customConfig, validateFunc); + validateInConfigWithExplicitTotalFlinkAndManagedMem(customConfig, validateFunc); + validateInConfigWithExplicitTotalProcessMem(customConfig, validateFunc); + } + + private void validateFailInAllConfigurations(final Configuration customConfig) { + validateFailInConfigWithExplicitTaskHeapAndManagedMem(customConfig); + validateFailInConfigWithExplicitTotalFlinkMem(customConfig); + validateFailInConfigWithExplicitTotalFlinkAndTaskHeapMem(customConfig); + validateFailInConfigWithExplicitTotalFlinkAndManagedMem(customConfig); + validateFailInConfigWithExplicitTotalProcessMem(customConfig); + } + + private void validateInConfigurationsWithoutExplicitTaskHeapMem(final Configuration customConfig, Consumer validateFunc) { + validateInConfigWithExplicitTotalFlinkMem(customConfig, validateFunc); + validateInConfigWithExplicitTotalFlinkAndManagedMem(customConfig, validateFunc); + validateInConfigWithExplicitTotalProcessMem(customConfig, validateFunc); + } + + private void validateInConfigurationsWithoutExplicitManagedMem(final Configuration customConfig, Consumer validateFunc) { + validateInConfigWithExplicitTotalFlinkMem(customConfig, validateFunc); + validateInConfigWithExplicitTotalFlinkAndTaskHeapMem(customConfig, validateFunc); + validateInConfigWithExplicitTotalProcessMem(customConfig, validateFunc); + } + + private void validateFailInConfigurationsWithoutExplicitManagedMem(final Configuration customConfig) { + validateFailInConfigWithExplicitTotalFlinkMem(customConfig); + validateFailInConfigWithExplicitTotalFlinkAndTaskHeapMem(customConfig); + validateFailInConfigWithExplicitTotalProcessMem(customConfig); + } + + private void validateInConfigWithExplicitTaskHeapAndManagedMem( + final Configuration customConfig, Consumer validateFunc) { + log.info("Validating in configuration with explicit task heap and managed memory size."); + final Configuration config = configWithExplicitTaskHeapAndManageMem(); + config.addAll(customConfig); + TaskExecutorResourceSpec taskExecutorResourceSpec = TaskExecutorResourceUtils.resourceSpecFromConfig(config); + assertThat(taskExecutorResourceSpec.getTaskHeapSize(), is(TASK_HEAP_SIZE)); + assertThat(taskExecutorResourceSpec.getManagedMemorySize(), is(MANAGED_MEM_SIZE)); + validateFunc.accept(taskExecutorResourceSpec); + } + + private void validateFailInConfigWithExplicitTaskHeapAndManagedMem(final Configuration customConfig) { + log.info("Validating failing in configuration with explicit task heap and managed memory size."); + final Configuration config = configWithExplicitTaskHeapAndManageMem(); + config.addAll(customConfig); + validateFail(config); + } + + private void validateInConfigWithExplicitTotalFlinkMem( + final Configuration customConfig, Consumer validateFunc) { + log.info("Validating in configuration with explicit total flink memory size."); + final Configuration config = configWithExplicitTotalFlinkMem(); + config.addAll(customConfig); + TaskExecutorResourceSpec taskExecutorResourceSpec = TaskExecutorResourceUtils.resourceSpecFromConfig(config); + assertThat(taskExecutorResourceSpec.getTotalFlinkMemorySize(), is(TOTAL_FLINK_MEM_SIZE)); + validateFunc.accept(taskExecutorResourceSpec); + } + + private void validateFailInConfigWithExplicitTotalFlinkMem(final Configuration customConfig) { + log.info("Validating failing in configuration with explicit total flink memory size."); + final Configuration config = configWithExplicitTotalFlinkMem(); + config.addAll(customConfig); + validateFail(config); + } + + private void validateInConfigWithExplicitTotalFlinkAndTaskHeapMem( + final Configuration customConfig, Consumer validateFunc) { + log.info("Validating in configuration with explicit total flink and task heap memory size."); + final Configuration config = configWithExplicitTotalFlinkAndTaskHeapMem(); + config.addAll(customConfig); + TaskExecutorResourceSpec taskExecutorResourceSpec = TaskExecutorResourceUtils.resourceSpecFromConfig(config); + assertThat(taskExecutorResourceSpec.getTotalFlinkMemorySize(), is(TOTAL_FLINK_MEM_SIZE)); + assertThat(taskExecutorResourceSpec.getTaskHeapSize(), is(TASK_HEAP_SIZE)); + validateFunc.accept(taskExecutorResourceSpec); + } + + private void validateFailInConfigWithExplicitTotalFlinkAndTaskHeapMem(final Configuration customConfig) { + log.info("Validating failing in configuration with explicit total flink and task heap memory size."); + final Configuration config = configWithExplicitTotalFlinkAndTaskHeapMem(); + config.addAll(customConfig); + validateFail(config); + } + + private void validateInConfigWithExplicitTotalFlinkAndManagedMem( + final Configuration customConfig, Consumer validateFunc) { + log.info("Validating in configuration with explicit total flink and managed memory size."); + final Configuration config = configWithExplicitTotalFlinkAndManagedMem(); + config.addAll(customConfig); + TaskExecutorResourceSpec taskExecutorResourceSpec = TaskExecutorResourceUtils.resourceSpecFromConfig(config); + assertThat(taskExecutorResourceSpec.getTotalFlinkMemorySize(), is(TOTAL_FLINK_MEM_SIZE)); + assertThat(taskExecutorResourceSpec.getManagedMemorySize(), is(MANAGED_MEM_SIZE)); + validateFunc.accept(taskExecutorResourceSpec); + } + + private void validateFailInConfigWithExplicitTotalFlinkAndManagedMem(final Configuration customConfig) { + log.info("Validating failing in configuration with explicit total flink and managed memory size."); + final Configuration config = configWithExplicitTotalFlinkAndManagedMem(); + config.addAll(customConfig); + validateFail(config); + } + + private void validateInConfigWithExplicitTotalProcessMem( + final Configuration customConfig, Consumer validateFunc) { + log.info("Validating in configuration with explicit total process memory size."); + final Configuration config = configWithExplicitTotalProcessMem(); + config.addAll(customConfig); + TaskExecutorResourceSpec taskExecutorResourceSpec = TaskExecutorResourceUtils.resourceSpecFromConfig(config); + assertThat(taskExecutorResourceSpec.getTotalProcessMemorySize(), is(TOTAL_PROCESS_MEM_SIZE)); + validateFunc.accept(taskExecutorResourceSpec); + } + + private void validateFailInConfigWithExplicitTotalProcessMem(final Configuration customConfig) { + log.info("Validating failing in configuration with explicit total process memory size."); + final Configuration config = configWithExplicitTotalProcessMem(); + config.addAll(customConfig); + validateFail(config); + } + + private void validateFail(final Configuration config) { + try { + TaskExecutorResourceUtils.resourceSpecFromConfig(config); + fail("Configuration did not fail as expected."); + } catch (Throwable t) { + // expected + } + } + + private static Configuration configWithExplicitTaskHeapAndManageMem() { + final Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.TASK_HEAP_MEMORY, TASK_HEAP_SIZE.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, MANAGED_MEM_SIZE.getMebiBytes() + "m"); + return conf; + } + + private static Configuration configWithExplicitTotalFlinkMem() { + final Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.TOTAL_FLINK_MEMORY, TOTAL_FLINK_MEM_SIZE.getMebiBytes() + "m"); + return conf; + } + + private static Configuration configWithExplicitTotalFlinkAndTaskHeapMem() { + final Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.TOTAL_FLINK_MEMORY, TOTAL_FLINK_MEM_SIZE.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.TASK_HEAP_MEMORY, TASK_HEAP_SIZE.getMebiBytes() + "m"); + return conf; + } + + private static Configuration configWithExplicitTotalFlinkAndManagedMem() { + final Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.TOTAL_FLINK_MEMORY, TOTAL_FLINK_MEM_SIZE.getMebiBytes() + "m"); + conf.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, MANAGED_MEM_SIZE.getMebiBytes() + "m"); + return conf; + } + + private static Configuration configWithExplicitTotalProcessMem() { + final Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.TOTAL_PROCESS_MEMORY, TOTAL_PROCESS_MEM_SIZE.getMebiBytes() + "m"); + return conf; + } +} -- Gitee From d571b2bcab8188ae12e47a63ea4cc5d4583fb7de Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Sat, 5 Oct 2019 15:28:50 +0800 Subject: [PATCH 212/268] [FLINK-13982][runtime] Generate dynamic configurations and JVM parameters with TaskExecutorResourceUtils. This closes #9760. --- .../TaskExecutorResourceUtils.java | 45 ++++++++++++ .../TaskExecutorResourceUtilsTest.java | 68 ++++++++++++++++++- 2 files changed, 111 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtils.java index ba521bb10c..ee13943a78 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtils.java @@ -25,6 +25,9 @@ import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.util.ConfigurationParserUtils; +import java.util.HashMap; +import java.util.Map; + import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -37,6 +40,48 @@ public class TaskExecutorResourceUtils { private TaskExecutorResourceUtils() {} + // ------------------------------------------------------------------------ + // Generating JVM Parameters + // ------------------------------------------------------------------------ + + public static String generateJvmParametersStr(final TaskExecutorResourceSpec taskExecutorResourceSpec) { + final MemorySize jvmHeapSize = taskExecutorResourceSpec.getFrameworkHeapSize() + .add(taskExecutorResourceSpec.getTaskHeapSize()) + .add(taskExecutorResourceSpec.getOnHeapManagedMemorySize()); + final MemorySize jvmDirectSize = taskExecutorResourceSpec.getTaskOffHeapSize() + .add(taskExecutorResourceSpec.getShuffleMemSize()); + final MemorySize jvmMetaspaceSize = taskExecutorResourceSpec.getJvmMetaspaceSize(); + + return "-Xmx" + jvmHeapSize.getBytes() + + " -Xms" + jvmHeapSize.getBytes() + + " -XX:MaxDirectMemorySize=" + jvmDirectSize.getBytes() + + " -XX:MaxMetaspaceSize=" + jvmMetaspaceSize.getBytes(); + } + + // ------------------------------------------------------------------------ + // Generating Dynamic Config Options + // ------------------------------------------------------------------------ + + public static String generateDynamicConfigsStr(final TaskExecutorResourceSpec taskExecutorResourceSpec) { + final Map configs = new HashMap<>(); + configs.put(TaskManagerOptions.FRAMEWORK_HEAP_MEMORY.key(), taskExecutorResourceSpec.getFrameworkHeapSize().getBytes() + "b"); + configs.put(TaskManagerOptions.TASK_HEAP_MEMORY.key(), taskExecutorResourceSpec.getTaskHeapSize().getBytes() + "b"); + configs.put(TaskManagerOptions.TASK_OFF_HEAP_MEMORY.key(), taskExecutorResourceSpec.getTaskOffHeapSize().getBytes() + "b"); + configs.put(TaskManagerOptions.SHUFFLE_MEMORY_MIN.key(), taskExecutorResourceSpec.getShuffleMemSize().getBytes() + "b"); + configs.put(TaskManagerOptions.SHUFFLE_MEMORY_MAX.key(), taskExecutorResourceSpec.getShuffleMemSize().getBytes() + "b"); + configs.put(TaskManagerOptions.MANAGED_MEMORY_SIZE.key(), taskExecutorResourceSpec.getManagedMemorySize().getBytes() + "b"); + configs.put(TaskManagerOptions.MANAGED_MEMORY_OFFHEAP_SIZE.key(), taskExecutorResourceSpec.getOffHeapManagedMemorySize().getBytes() + "b"); + return assembleDynamicConfigsStr(configs); + } + + private static String assembleDynamicConfigsStr(final Map configs) { + final StringBuilder sb = new StringBuilder(); + for (Map.Entry entry : configs.entrySet()) { + sb.append("-D ").append(entry.getKey()).append("=").append(entry.getValue()).append(" "); + } + return sb.toString(); + } + // ------------------------------------------------------------------------ // Memory Configuration Calculations // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtilsTest.java index a77370d38e..8577d8f1c8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtilsTest.java @@ -27,6 +27,8 @@ import org.apache.flink.util.TestLogger; import org.junit.Test; +import java.util.HashMap; +import java.util.Map; import java.util.function.Consumer; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -45,8 +47,70 @@ public class TaskExecutorResourceUtilsTest extends TestLogger { private static final MemorySize TOTAL_FLINK_MEM_SIZE = MemorySize.parse("800m"); private static final MemorySize TOTAL_PROCESS_MEM_SIZE = MemorySize.parse("1g"); - @Test - public void testConfigFrameworkHeapMemory() { + private static final TaskExecutorResourceSpec TM_RESOURCE_SPEC = new TaskExecutorResourceSpec( + MemorySize.parse("1m"), + MemorySize.parse("2m"), + MemorySize.parse("3m"), + MemorySize.parse("4m"), + MemorySize.parse("5m"), + MemorySize.parse("6m"), + MemorySize.parse("7m"), + MemorySize.parse("8m")); + + @Test + public void testGenerateDynamicConfigurations() { + String dynamicConfigsStr = TaskExecutorResourceUtils.generateDynamicConfigsStr(TM_RESOURCE_SPEC); + Map configs = new HashMap<>(); + String[] configStrs = dynamicConfigsStr.split(" "); + assertThat(configStrs.length % 2, is(0)); + for (int i = 0; i < configStrs.length; ++i) { + String configStr = configStrs[i]; + if (i % 2 == 0) { + assertThat(configStr, is("-D")); + } else { + String[] configKV = configStr.split("="); + assertThat(configKV.length, is(2)); + configs.put(configKV[0], configKV[1]); + } + } + + assertThat(MemorySize.parse(configs.get(TaskManagerOptions.FRAMEWORK_HEAP_MEMORY.key())), is(TM_RESOURCE_SPEC.getFrameworkHeapSize())); + assertThat(MemorySize.parse(configs.get(TaskManagerOptions.TASK_HEAP_MEMORY.key())), is(TM_RESOURCE_SPEC.getTaskHeapSize())); + assertThat(MemorySize.parse(configs.get(TaskManagerOptions.TASK_OFF_HEAP_MEMORY.key())), is(TM_RESOURCE_SPEC.getTaskOffHeapSize())); + assertThat(MemorySize.parse(configs.get(TaskManagerOptions.SHUFFLE_MEMORY_MAX.key())), is(TM_RESOURCE_SPEC.getShuffleMemSize())); + assertThat(MemorySize.parse(configs.get(TaskManagerOptions.SHUFFLE_MEMORY_MIN.key())), is(TM_RESOURCE_SPEC.getShuffleMemSize())); + assertThat(MemorySize.parse(configs.get(TaskManagerOptions.MANAGED_MEMORY_SIZE.key())), is(TM_RESOURCE_SPEC.getManagedMemorySize())); + assertThat(MemorySize.parse(configs.get(TaskManagerOptions.MANAGED_MEMORY_OFFHEAP_SIZE.key())), is(TM_RESOURCE_SPEC.getOffHeapManagedMemorySize())); + } + + @Test + public void testGenerateJvmParameters() throws Exception { + String jvmParamsStr = TaskExecutorResourceUtils.generateJvmParametersStr(TM_RESOURCE_SPEC); + MemorySize heapSizeMax = null; + MemorySize heapSizeMin = null; + MemorySize directSize = null; + MemorySize metaspaceSize = null; + for (String paramStr : jvmParamsStr.split(" ")) { + if (paramStr.startsWith("-Xmx")) { + heapSizeMax = MemorySize.parse(paramStr.substring("-Xmx".length())); + } else if (paramStr.startsWith("-Xms")) { + heapSizeMin = MemorySize.parse(paramStr.substring("-Xms".length())); + } else if (paramStr.startsWith("-XX:MaxDirectMemorySize=")) { + directSize = MemorySize.parse(paramStr.substring("-XX:MaxDirectMemorySize=".length())); + } else if (paramStr.startsWith("-XX:MaxMetaspaceSize=")) { + metaspaceSize = MemorySize.parse(paramStr.substring("-XX:MaxMetaspaceSize=".length())); + } else { + throw new Exception("Unknown JVM parameter: " + paramStr); + } + } + + assertThat(heapSizeMax, is(TM_RESOURCE_SPEC.getFrameworkHeapSize().add(TM_RESOURCE_SPEC.getTaskHeapSize()).add(TM_RESOURCE_SPEC.getOnHeapManagedMemorySize()))); + assertThat(heapSizeMin, is(heapSizeMax)); + assertThat(directSize, is(TM_RESOURCE_SPEC.getTaskOffHeapSize().add(TM_RESOURCE_SPEC.getShuffleMemSize()))); + assertThat(metaspaceSize, is(TM_RESOURCE_SPEC.getJvmMetaspaceSize())); + } + + @Test public void testConfigFrameworkHeapMemory() { final MemorySize frameworkHeapSize = MemorySize.parse("100m"); Configuration conf = new Configuration(); -- Gitee From 4966b1fce1c22fe96c740fe979e96e1b78c434f0 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 14 Oct 2019 12:16:10 +0200 Subject: [PATCH 213/268] [FLINK-12576][docs,metrics] Document that input pool usage metrics ignore LocalInputChannels --- docs/monitoring/metrics.md | 6 +++--- docs/monitoring/metrics.zh.md | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index 70fdcc5cf3..34336ff5c9 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -1046,17 +1046,17 @@ Thus, in order to infer the metric identifier: inPoolUsage - An estimate of the input buffers usage. + An estimate of the input buffers usage. (ignores LocalInputChannels) Gauge inputFloatingBuffersUsage - An estimate of the floating input buffers usage, dediciated for credit-based mode. + An estimate of the floating input buffers usage, dediciated for credit-based mode. (ignores LocalInputChannels) Gauge inputExclusiveBuffersUsage - An estimate of the exclusive input buffers usage, dediciated for credit-based mode. + An estimate of the exclusive input buffers usage, dediciated for credit-based mode. (ignores LocalInputChannels) Gauge diff --git a/docs/monitoring/metrics.zh.md b/docs/monitoring/metrics.zh.md index fa8e14e2f8..aac82cc395 100644 --- a/docs/monitoring/metrics.zh.md +++ b/docs/monitoring/metrics.zh.md @@ -1046,17 +1046,17 @@ Thus, in order to infer the metric identifier: inPoolUsage - An estimate of the input buffers usage. + An estimate of the input buffers usage. (ignores LocalInputChannels) Gauge inputFloatingBuffersUsage - An estimate of the floating input buffers usage, dediciated for credit-based mode. + An estimate of the floating input buffers usage, dediciated for credit-based mode. (ignores LocalInputChannels) Gauge inputExclusiveBuffersUsage - An estimate of the exclusive input buffers usage, dediciated for credit-based mode. + An estimate of the exclusive input buffers usage, dediciated for credit-based mode. (ignores LocalInputChannels) Gauge -- Gitee From 73ec89be5c7cb07c0d4f6d2352c7182341ff71d5 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 14 Oct 2019 12:19:22 +0200 Subject: [PATCH 214/268] [hotfix][docs,metrics] Fix typo in the input pool usage metrics --- docs/monitoring/metrics.md | 4 ++-- docs/monitoring/metrics.zh.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index 34336ff5c9..696b508ae4 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -1051,12 +1051,12 @@ Thus, in order to infer the metric identifier: inputFloatingBuffersUsage - An estimate of the floating input buffers usage, dediciated for credit-based mode. (ignores LocalInputChannels) + An estimate of the floating input buffers usage, dedicated for credit-based mode. (ignores LocalInputChannels) Gauge inputExclusiveBuffersUsage - An estimate of the exclusive input buffers usage, dediciated for credit-based mode. (ignores LocalInputChannels) + An estimate of the exclusive input buffers usage, dedicated for credit-based mode. (ignores LocalInputChannels) Gauge diff --git a/docs/monitoring/metrics.zh.md b/docs/monitoring/metrics.zh.md index aac82cc395..389c65cabf 100644 --- a/docs/monitoring/metrics.zh.md +++ b/docs/monitoring/metrics.zh.md @@ -1051,12 +1051,12 @@ Thus, in order to infer the metric identifier: inputFloatingBuffersUsage - An estimate of the floating input buffers usage, dediciated for credit-based mode. (ignores LocalInputChannels) + An estimate of the floating input buffers usage, dedicated for credit-based mode. (ignores LocalInputChannels) Gauge inputExclusiveBuffersUsage - An estimate of the exclusive input buffers usage, dediciated for credit-based mode. (ignores LocalInputChannels) + An estimate of the exclusive input buffers usage, dedicated for credit-based mode. (ignores LocalInputChannels) Gauge -- Gitee From 34379182d63991baf9672c48842f06316d541170 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 30 Sep 2019 13:41:47 +0800 Subject: [PATCH 215/268] [FLINK-14246][runtime] Annotate MiniClusterITCase with AlsoRunWithSchedulerNG and fix broken tests The tests broke because the error messages of NoResourceAvailableExceptions from LegacyScheduler and DefaultScheduler are different. --- .../runtime/minicluster/MiniClusterITCase.java | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) 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 3744f11618..6287815409 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 @@ -46,9 +46,11 @@ import org.apache.flink.runtime.jobmaster.TestingAbstractInvokables.Sender; import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testtasks.WaitingNoOpInvokable; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.TestLogger; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.io.IOException; import java.util.concurrent.CompletableFuture; @@ -62,6 +64,7 @@ import static org.junit.Assert.fail; /** * Integration test cases for the {@link MiniCluster}. */ +@Category(AlsoRunWithSchedulerNG.class) public class MiniClusterITCase extends TestLogger { @Test @@ -110,7 +113,12 @@ public class MiniClusterITCase extends TestLogger { } 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()); + + //TODO: remove the legacy scheduler message check once legacy scheduler is removed + final String legacySchedulerErrorMessage = "Slots required: 2, slots allocated: 1"; + final String ngSchedulerErrorMessage = "Could not allocate the required slot within slot request timeout"; + assertTrue(findThrowableWithMessage(e, legacySchedulerErrorMessage).isPresent() || + findThrowableWithMessage(e, ngSchedulerErrorMessage).isPresent()); } } @@ -122,7 +130,12 @@ public class MiniClusterITCase extends TestLogger { } 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()); + + //TODO: remove the legacy scheduler message check once legacy scheduler is removed + final String legacySchedulerErrorMessage = "Could not allocate enough slots"; + final String ngSchedulerErrorMessage = "Could not allocate the required slot within slot request timeout"; + assertTrue(findThrowableWithMessage(e, legacySchedulerErrorMessage).isPresent() || + findThrowableWithMessage(e, ngSchedulerErrorMessage).isPresent()); } } -- Gitee From eee2988b3b1be37c4c465fb75da06dea7e46328d Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 30 Sep 2019 14:19:10 +0800 Subject: [PATCH 216/268] [FLINK-14246][runtime] Annotate TaskExecutorITCase with AlsoRunWithSchedulerNG and fix broken tests The test testJobRecoveryWithFailingTaskExecutor was to fail because terminating one TM will cause 2 tasks to fail, leading to 2 failure recoveries when using DefaultScheduler. And the restart strategy limited max failure count to 1. --- .../flink/runtime/taskexecutor/TaskExecutorITCase.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java index 71c2775374..630bb8ef51 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java @@ -36,12 +36,14 @@ import org.apache.flink.runtime.jobmaster.TestingAbstractInvokables; import org.apache.flink.runtime.minicluster.TestingMiniCluster; import org.apache.flink.runtime.minicluster.TestingMiniClusterConfiguration; import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.SupplierWithException; import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.io.IOException; import java.time.Duration; @@ -56,6 +58,7 @@ import static org.junit.Assert.assertThat; /** * Integration tests for the {@link TaskExecutor}. */ +@Category(AlsoRunWithSchedulerNG.class) public class TaskExecutorITCase extends TestLogger { private static final Duration TESTING_TIMEOUT = Duration.ofMinutes(2L); @@ -157,7 +160,7 @@ public class TaskExecutorITCase extends TestLogger { private JobGraph createJobGraphWithRestartStrategy(int parallelism) throws IOException { final JobGraph jobGraph = createJobGraph(parallelism); final ExecutionConfig executionConfig = new ExecutionConfig(); - executionConfig.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0L)); + executionConfig.setRestartStrategy(RestartStrategies.fixedDelayRestart(2, 0L)); jobGraph.setExecutionConfig(executionConfig); return jobGraph; -- Gitee From dafc6192a911cf22da669069d8644d30082a1c96 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 30 Sep 2019 14:21:08 +0800 Subject: [PATCH 217/268] [FLINK-14246][runtime] Annotate all other MiniCluster tests in flink-runtime with AlsoRunWithSchedulerNG This closes #9901. --- .../network/partition/PartialConsumePipelinedResultTest.java | 3 +++ .../apache/flink/runtime/jobmanager/BlobsCleanupITCase.java | 3 +++ .../runtime/jobmanager/SlotCountExceedingParallelismTest.java | 3 +++ .../jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java | 3 +++ .../org/apache/flink/runtime/jobmaster/JobExecutionITCase.java | 3 +++ .../org/apache/flink/runtime/jobmaster/JobRecoveryITCase.java | 3 +++ .../leaderelection/LeaderChangeClusterComponentsTest.java | 3 +++ .../taskmanager/TaskCancelAsyncProducerConsumerITCase.java | 3 +++ 8 files changed, 24 insertions(+) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java index 12ab07754f..fa8373544a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java @@ -34,14 +34,17 @@ import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.MiniClusterResource; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; /** * Test for consuming a pipelined result only partially. */ +@Category(AlsoRunWithSchedulerNG.class) public class PartialConsumePipelinedResultTest extends TestLogger { // Test configuration diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/BlobsCleanupITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/BlobsCleanupITCase.java index bb2aede627..bbc57333b8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/BlobsCleanupITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/BlobsCleanupITCase.java @@ -38,6 +38,7 @@ import org.apache.flink.runtime.testtasks.FailingBlockingInvokable; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.MiniClusterResource; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -45,6 +46,7 @@ import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import javax.annotation.Nonnull; @@ -69,6 +71,7 @@ import static org.junit.Assert.fail; * Small test to check that the {@link org.apache.flink.runtime.blob.BlobServer} cleanup is executed * after job termination. */ +@Category(AlsoRunWithSchedulerNG.class) public class BlobsCleanupITCase extends TestLogger { private static final long RETRY_INTERVAL = 100L; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java index 5f3e7081ae..8e6a22f0f9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java @@ -33,11 +33,13 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.MiniClusterResource; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.types.IntValue; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.util.BitSet; @@ -46,6 +48,7 @@ import java.util.BitSet; * of slots. This effectively tests that Flink can execute jobs with blocking results * in a staged fashion. */ +@Category(AlsoRunWithSchedulerNG.class) public class SlotCountExceedingParallelismTest extends TestLogger { // Test configuration diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java index 20ce3b3919..cbd2fae49f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.jobmanager.SlotCountExceedingParallelismTest; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.MiniClusterResource; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.types.IntValue; import org.apache.flink.util.TestLogger; @@ -39,6 +40,7 @@ import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.util.List; @@ -48,6 +50,7 @@ import static org.apache.flink.runtime.jobmanager.SlotCountExceedingParallelismT * Tests for the lazy scheduling/updating of consumers depending on the * producers result. */ +@Category(AlsoRunWithSchedulerNG.class) public class ScheduleOrUpdateConsumersTest extends TestLogger { private static final int NUMBER_OF_TMS = 2; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobExecutionITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobExecutionITCase.java index e19fe2020d..03f36b8029 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobExecutionITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobExecutionITCase.java @@ -25,9 +25,11 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.minicluster.TestingMiniCluster; import org.apache.flink.runtime.minicluster.TestingMiniClusterConfiguration; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.TestLogger; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.util.concurrent.CompletableFuture; @@ -37,6 +39,7 @@ import static org.junit.Assert.assertThat; /** * Integration tests for job scheduling. */ +@Category(AlsoRunWithSchedulerNG.class) public class JobExecutionITCase extends TestLogger { /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobRecoveryITCase.java index d003912eae..d9f38d370d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobRecoveryITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobRecoveryITCase.java @@ -29,11 +29,13 @@ import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.testutils.MiniClusterResource; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.io.IOException; import java.util.concurrent.CompletableFuture; @@ -44,6 +46,7 @@ import static org.junit.Assert.assertThat; /** * Tests for the recovery of task failures. */ +@Category(AlsoRunWithSchedulerNG.class) public class JobRecoveryITCase extends TestLogger { private static final int NUM_TMS = 1; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java index 0731d96fee..fbb1bad9f3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.minicluster.TestingMiniClusterConfiguration; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.util.LeaderRetrievalUtils; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -40,6 +41,7 @@ import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.time.Duration; import java.util.concurrent.CompletableFuture; @@ -53,6 +55,7 @@ import static org.junit.Assert.fail; /** * Tests which verify the cluster behaviour in case of leader changes. */ +@Category(AlsoRunWithSchedulerNG.class) public class LeaderChangeClusterComponentsTest extends TestLogger { private static final Duration TESTING_TIMEOUT = Duration.ofMinutes(2L); 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 0d5588e78b..e3fda33f50 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 @@ -40,11 +40,13 @@ import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.MiniClusterResource; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.types.LongValue; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.time.Duration; import java.util.Arrays; @@ -55,6 +57,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +@Category(AlsoRunWithSchedulerNG.class) public class TaskCancelAsyncProducerConsumerITCase extends TestLogger { // The Exceptions thrown by the producer/consumer Threads -- Gitee From ca5e51822e5139702dae3e85f9237cdb41d4ff70 Mon Sep 17 00:00:00 2001 From: hpeter Date: Sat, 12 Oct 2019 16:45:34 -0700 Subject: [PATCH 218/268] [FLINK-14215][docs] Add how to configure environment variables to documentation This closes #9887. --- docs/ops/config.md | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/ops/config.md b/docs/ops/config.md index a680a0a230..5a36efb0cc 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -231,7 +231,6 @@ You have to configure `jobmanager.archive.fs.dir` in order to archive terminated ## Background - ### Configuring the Network Buffers If you ever see the Exception `java.io.IOException: Insufficient number of network buffers`, you @@ -319,4 +318,12 @@ When starting a Flink application, users can supply the default number of slots +### Configuration Runtime Environment Variables +You have to set config with prefix `containerized.master.env.` and `containerized.taskmanager.env.` in order to set redefined environment variable in ApplicationMaster and TaskManager. + +- `containerized.master.env.`: Prefix for passing custom environment variables to Flink's master process. + For example for passing LD_LIBRARY_PATH as an env variable to the AppMaster, set containerized.master.env.LD_LIBRARY_PATH: "/usr/lib/native" + in the flink-conf.yaml. +- `containerized.taskmanager.env.`: Similar to the above, this configuration prefix allows setting custom environment variables for the workers (TaskManagers). + {% top %} -- Gitee From 60cc18bb6ec20a66ca79b835dc67b5cdbfb3c881 Mon Sep 17 00:00:00 2001 From: liyafan82 Date: Mon, 1 Jul 2019 18:13:38 +0800 Subject: [PATCH 219/268] [FLINK-12628][Runtime / Coordination] Remove no consumers check in Execution.getPartitionMaxParallelism Currently, we have a TODO for this case in Execution.getPartitionMaxParallelism because of tests: // TODO consumers.isEmpty() only exists for test, currently there has to be exactly one consumer in real jobs! though partition is supposed to have always at least one consumer at the moment. After having run the CI, there is no test failure, when we ignore the case for consumers.isEmpty() equals to true. This means we can remove the TODO and the check leaving only the precondition assertion. --- .../flink/runtime/executiongraph/Execution.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 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 503b59e288..d9cdf5a9e4 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 @@ -57,7 +57,6 @@ import org.apache.flink.runtime.shuffle.PartitionDescriptor; import org.apache.flink.runtime.shuffle.ProducerDescriptor; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleMaster; -import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -657,14 +656,11 @@ public class Execution implements AccessExecution, Archiveable> consumers = partition.getConsumers(); - int maxParallelism = KeyGroupRangeAssignment.UPPER_BOUND_MAX_PARALLELISM; - if (!consumers.isEmpty()) { - List consumer = consumers.get(0); - ExecutionJobVertex consumerVertex = consumer.get(0).getTarget().getJobVertex(); - maxParallelism = consumerVertex.getMaxParallelism(); - } + Preconditions.checkArgument(!consumers.isEmpty(), "Currently there has to be exactly one consumer in real jobs"); + List consumer = consumers.get(0); + ExecutionJobVertex consumerVertex = consumer.get(0).getTarget().getJobVertex(); + int maxParallelism = consumerVertex.getMaxParallelism(); return maxParallelism; } -- Gitee From 2894caf4a2a054a4838af5b631d40d15c43369ff Mon Sep 17 00:00:00 2001 From: "kevin.cyj" Date: Tue, 24 Sep 2019 18:23:56 +0800 Subject: [PATCH 220/268] [FLINK-14185][tests]Move unstable assertion statement out of close method of QS test server. --- .../network/AbstractServerTest.java | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java index 9e5720d155..02bfa7cf09 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java @@ -86,8 +86,8 @@ public class AbstractServerTest extends TestLogger { @Test public void testPortRangeSuccess() throws Throwable { - // this is shared between the two servers. - AtomicKvStateRequestStats serverStats = new AtomicKvStateRequestStats(); + AtomicKvStateRequestStats serverStats1 = new AtomicKvStateRequestStats(); + AtomicKvStateRequestStats serverStats2 = new AtomicKvStateRequestStats(); AtomicKvStateRequestStats clientStats = new AtomicKvStateRequestStats(); final int portRangeStart = 7777; @@ -95,8 +95,8 @@ public class AbstractServerTest extends TestLogger { List portList = IntStream.range(portRangeStart, portRangeEnd + 1).boxed().collect(Collectors.toList()); try ( - TestServer server1 = new TestServer("Test Server 1", serverStats, portList.iterator()); - TestServer server2 = new TestServer("Test Server 2", serverStats, portList.iterator()); + TestServer server1 = new TestServer("Test Server 1", serverStats1, portList.iterator()); + TestServer server2 = new TestServer("Test Server 2", serverStats2, portList.iterator()); TestClient client = new TestClient( "Test Client", 1, @@ -116,8 +116,8 @@ public class AbstractServerTest extends TestLogger { TestMessage response2 = client.sendRequest(server2.getServerAddress(), new TestMessage("pong")).join(); Assert.assertEquals(server2.getServerName() + "-pong", response2.getMessage()); - // the client connects to both servers and the stats object is shared. - Assert.assertEquals(2L, serverStats.getNumConnections()); + Assert.assertEquals(1L, serverStats1.getNumConnections()); + Assert.assertEquals(1L, serverStats2.getNumConnections()); Assert.assertEquals(2L, clientStats.getNumConnections()); Assert.assertEquals(0L, clientStats.getNumFailed()); @@ -125,6 +125,9 @@ public class AbstractServerTest extends TestLogger { Assert.assertEquals(2L, clientStats.getNumRequests()); } + Assert.assertEquals(0L, serverStats1.getNumConnections()); + Assert.assertEquals(0L, serverStats2.getNumConnections()); + Assert.assertEquals(0L, clientStats.getNumConnections()); Assert.assertEquals(0L, clientStats.getNumFailed()); Assert.assertEquals(2L, clientStats.getNumSuccessful()); @@ -184,10 +187,6 @@ public class AbstractServerTest extends TestLogger { @Override public void close() throws Exception { shutdownServer().get(); - if (requestStats instanceof AtomicKvStateRequestStats) { - AtomicKvStateRequestStats stats = (AtomicKvStateRequestStats) requestStats; - Assert.assertEquals(0L, stats.getNumConnections()); - } Assert.assertTrue(getQueryExecutor().isTerminated()); Assert.assertTrue(isEventGroupShutdown()); } -- Gitee From 3dcc72b29834aae859cd053af7bd01e5162d5727 Mon Sep 17 00:00:00 2001 From: huzheng Date: Wed, 9 Oct 2019 17:36:56 +0800 Subject: [PATCH 221/268] [FLINK-14349][hbase] Create a Connector Descriptor for HBase so that user can connect HBase by TableEnvironment#connect This closes #9866 --- flink-connectors/flink-hbase/pom.xml | 7 + .../apache/flink/table/descriptors/HBase.java | 129 ++++++++++++++++++ .../addons/hbase/HBaseDescriptorTest.java | 120 ++++++++++++++++ .../example/HBaseFlinkTestConstants.java | 6 +- 4 files changed, 259 insertions(+), 3 deletions(-) create mode 100644 flink-connectors/flink-hbase/src/main/java/org/apache/flink/table/descriptors/HBase.java create mode 100644 flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseDescriptorTest.java diff --git a/flink-connectors/flink-hbase/pom.xml b/flink-connectors/flink-hbase/pom.xml index 10bffabf36..d5237fc504 100644 --- a/flink-connectors/flink-hbase/pom.xml +++ b/flink-connectors/flink-hbase/pom.xml @@ -287,6 +287,13 @@ under the License. ${project.version} test + + org.apache.flink + flink-table-common + ${project.version} + test-jar + test + org.apache.flink flink-table-planner_${scala.binary.version} diff --git a/flink-connectors/flink-hbase/src/main/java/org/apache/flink/table/descriptors/HBase.java b/flink-connectors/flink-hbase/src/main/java/org/apache/flink/table/descriptors/HBase.java new file mode 100644 index 0000000000..66584f9b84 --- /dev/null +++ b/flink-connectors/flink-hbase/src/main/java/org/apache/flink/table/descriptors/HBase.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.table.descriptors; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.util.TimeUtils; + +import java.util.Map; + +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; +import static org.apache.flink.table.descriptors.HBaseValidator.CONNECTOR_TABLE_NAME; +import static org.apache.flink.table.descriptors.HBaseValidator.CONNECTOR_TYPE_VALUE_HBASE; +import static org.apache.flink.table.descriptors.HBaseValidator.CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL; +import static org.apache.flink.table.descriptors.HBaseValidator.CONNECTOR_WRITE_BUFFER_FLUSH_MAX_ROWS; +import static org.apache.flink.table.descriptors.HBaseValidator.CONNECTOR_WRITE_BUFFER_FLUSH_MAX_SIZE; +import static org.apache.flink.table.descriptors.HBaseValidator.CONNECTOR_ZK_NODE_PARENT; +import static org.apache.flink.table.descriptors.HBaseValidator.CONNECTOR_ZK_QUORUM; + +/** + * Connector descriptor for Apache HBase. + */ +@PublicEvolving +public class HBase extends ConnectorDescriptor { + private DescriptorProperties properties = new DescriptorProperties(); + + public HBase() { + super(CONNECTOR_TYPE_VALUE_HBASE, 1, true); + } + + /** + * Set the Apache HBase version to be used. Required. + * + * @param version HBase version. E.g., "1.4.3". + */ + public HBase version(String version) { + properties.putString(CONNECTOR_VERSION, version); + return this; + } + + /** + * Set the HBase table name, Required. + * + * @param tableName Name of HBase table. E.g., "testNamespace:testTable", "testDefaultTable" + */ + public HBase tableName(String tableName) { + properties.putString(CONNECTOR_TABLE_NAME, tableName); + return this; + } + + /** + * Set the zookeeper quorum address to connect the HBase cluster. Required. + * + * @param zookeeperQuorum zookeeper quorum address to connect the HBase cluster. + * E.g., "localhost:2181,localhost:2182,localhost:2183". + */ + public HBase zookeeperQuorum(String zookeeperQuorum) { + properties.putString(CONNECTOR_ZK_QUORUM, zookeeperQuorum); + return this; + } + + /** + * Set the zookeeper node parent path of HBase cluster. Default to use "/hbase", Optional. + * + * @param zookeeperNodeParent zookeeper node path of hbase cluster. E.g, "/hbase/example-root-znode". + */ + public HBase zookeeperNodeParent(String zookeeperNodeParent) { + properties.putString(CONNECTOR_ZK_NODE_PARENT, zookeeperNodeParent); + return this; + } + + /** + * Set threshold when to flush buffered request based on the memory byte size of rows currently added. + * Default to 2mb. Optional. + * + * @param maxSize the maximum size (using the syntax of {@link MemorySize}). + */ + public HBase writeBufferFlushMaxSize(String maxSize) { + properties.putMemorySize(CONNECTOR_WRITE_BUFFER_FLUSH_MAX_SIZE, MemorySize.parse(maxSize, MemorySize.MemoryUnit.BYTES)); + return this; + } + + /** + * Set threshold when to flush buffered request based on the number of rows currently added. + * Defaults to not set, i.e. won't flush based on the number of buffered rows. Optional. + * + * @param writeBufferFlushMaxRows number of added rows when begin the request flushing. + */ + public HBase writeBufferFlushMaxRows(int writeBufferFlushMaxRows) { + properties.putInt(CONNECTOR_WRITE_BUFFER_FLUSH_MAX_ROWS, writeBufferFlushMaxRows); + return this; + } + + /** + * Set a flush interval flushing buffered requesting if the interval passes, in milliseconds. + * Defaults to not set, i.e. won't flush based on flush interval. Optional. + * + * @param interval flush interval. The string should be in format "{length value}{time unit label}" + * E.g, "123ms", "1 s", If no time unit label is specified, it will be considered as + * milliseconds. For more details about the format, please see + * {@link TimeUtils#parseDuration(String)}}. + */ + public HBase writeBufferFlushInterval(String interval) { + Duration duration = TimeUtils.parseDuration(interval); + properties.putLong(CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL, duration.toMillis()); + return this; + } + + @Override + protected Map toConnectorProperties() { + return properties.asMap(); + } +} diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseDescriptorTest.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseDescriptorTest.java new file mode 100644 index 0000000000..b9a2f5dc2a --- /dev/null +++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseDescriptorTest.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.addons.hbase; + +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.descriptors.Descriptor; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.DescriptorTestBase; +import org.apache.flink.table.descriptors.DescriptorValidator; +import org.apache.flink.table.descriptors.HBase; +import org.apache.flink.table.descriptors.HBaseValidator; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Test case for {@link HBase} descriptor. + */ +public class HBaseDescriptorTest extends DescriptorTestBase { + + @Override + protected List descriptors() { + HBase hbaseDesc0 = new HBase() + .version("1.4.3") + .tableName("testNs:table0") + .zookeeperQuorum("localhost:2181,localhost:2182,localhost:2183") + .zookeeperNodeParent("/hbase/root-dir"); + + HBase hbaseDesc1 = new HBase() + .version("1.4.3") + .tableName("testNs:table1") + .zookeeperQuorum("localhost:2181") + .zookeeperNodeParent("/hbase/root") + .writeBufferFlushInterval("2s") + .writeBufferFlushMaxRows(100) + .writeBufferFlushMaxSize("1mb"); + + return Arrays.asList(hbaseDesc0, hbaseDesc1); + } + + @Override + protected List> properties() { + Map prop0 = new HashMap<>(); + prop0.put("connector.version", "1.4.3"); + prop0.put("connector.type", "hbase"); + prop0.put("connector.table-name", "testNs:table0"); + prop0.put("connector.zookeeper.quorum", "localhost:2181,localhost:2182,localhost:2183"); + prop0.put("connector.zookeeper.znode.parent", "/hbase/root-dir"); + prop0.put("connector.property-version", "1"); + + Map prop1 = new HashMap<>(); + prop1.put("connector.version", "1.4.3"); + prop1.put("connector.type", "hbase"); + prop1.put("connector.table-name", "testNs:table1"); + prop1.put("connector.zookeeper.quorum", "localhost:2181"); + prop1.put("connector.zookeeper.znode.parent", "/hbase/root"); + prop1.put("connector.property-version", "1"); + prop1.put("connector.write.buffer-flush.interval", "2s"); + prop1.put("connector.write.buffer-flush.max-rows", "100"); + prop1.put("connector.write.buffer-flush.max-size", "1048576 bytes"); + + return Arrays.asList(prop0, prop1); + } + + @Override + protected DescriptorValidator validator() { + return new HBaseValidator(); + } + + @Test + public void testRequiredFields() { + HBase hbaseDesc0 = new HBase(); + HBase hbaseDesc1 = new HBase() + .version("1.4.3") + .zookeeperQuorum("localhost:2181") + .zookeeperNodeParent("/hbase/root"); // no table name + HBase hbaseDesc2 = new HBase() + .version("1.4.3") + .tableName("ns:table") + .zookeeperNodeParent("/hbase/root"); // no zookeeper quorum + HBase hbaseDesc3 = new HBase() + .tableName("ns:table") + .zookeeperQuorum("localhost:2181"); // no version + + HBase[] testCases = new HBase[]{hbaseDesc0, hbaseDesc1, hbaseDesc2, hbaseDesc3}; + for (int i = 0; i < testCases.length; i++) { + HBase hbaseDesc = testCases[i]; + DescriptorProperties properties = new DescriptorProperties(); + properties.putProperties(hbaseDesc.toProperties()); + boolean caughtExpectedException = false; + try { + validator().validate(properties); + } catch (ValidationException e) { + caughtExpectedException = true; + } + Assert.assertTrue("The case#" + i + " didn't get the expected error", caughtExpectedException); + } + } +} diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseFlinkTestConstants.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseFlinkTestConstants.java index 57224c21a8..896dbbe1c7 100644 --- a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseFlinkTestConstants.java +++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseFlinkTestConstants.java @@ -18,12 +18,12 @@ package org.apache.flink.addons.hbase.example; -import org.apache.flink.configuration.ConfigConstants; +import org.apache.hadoop.hbase.util.Bytes; class HBaseFlinkTestConstants { - static final byte[] CF_SOME = "someCf".getBytes(ConfigConstants.DEFAULT_CHARSET); - static final byte[] Q_SOME = "someQual".getBytes(ConfigConstants.DEFAULT_CHARSET); + static final byte[] CF_SOME = Bytes.toBytes("someCf"); + static final byte[] Q_SOME = Bytes.toBytes("someQual"); static final String TEST_TABLE_NAME = "test-table"; static final String TMP_DIR = "/tmp/test"; -- Gitee From 95aec8604e7d950dbb3a2cd1f02a854428d4c61c Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 15 Oct 2019 21:01:30 +0800 Subject: [PATCH 222/268] [FLINK-14349][docs] Add documentation for HBase descriptor API --- docs/dev/table/connect.md | 25 ++++++++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/docs/dev/table/connect.md b/docs/dev/table/connect.md index f4f73e0685..975a0d01d8 100644 --- a/docs/dev/table/connect.md +++ b/docs/dev/table/connect.md @@ -1094,11 +1094,32 @@ For append-only queries, the connector can also operate in [append mode](#update The connector can be defined as follows:

    +
    +{% highlight java %} +.connect( + new HBase() + .version("1.4.3") // required: currently only support "1.4.3" + .tableName("hbase_table_name") // required: HBase table name + .zookeeperQuorum("localhost:2181") // required: HBase Zookeeper quorum configuration + .zookeeperNodeParent("/test") // optional: the root dir in Zookeeper for HBase cluster. + // The default value is "/hbase". + .writeBufferFlushMaxSize("10mb") // optional: writing option, determines how many size in memory of buffered + // rows to insert per round trip. This can help performance on writing to JDBC + // database. The default value is "2mb". + .writeBufferFlushMaxRows(1000) // optional: writing option, determines how many rows to insert per round trip. + // This can help performance on writing to JDBC database. No default value, + // i.e. the default flushing is not depends on the number of buffered rows. + .writeBufferFlushInterval("2s") // optional: writing option, sets a flush interval flushing buffered requesting + // if the interval passes, in milliseconds. Default value is "0s", which means + // no asynchronous flush thread will be scheduled. +) +{% endhighlight %} +
    {% highlight yaml %} connector: type: hbase - version: "1.4.3" # required: currently only support "1.4.3" + version: "1.4.3" # required: currently only support "1.4.3" table-name: "hbase_table_name" # required: HBase table name @@ -1157,8 +1178,6 @@ CREATE TABLE MyUserTable ( **Temporary join:** Lookup join against HBase do not use any caching; data is always queired directly through the HBase client. -**Java/Scala/Python API:** Java/Scala/Python APIs are not supported yet. - {% top %} ### JDBC Connector -- Gitee From 4fe51c1922dd005e508e74128140d5a7b0abab3c Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 15 Oct 2019 21:01:59 +0800 Subject: [PATCH 223/268] [hotfix][kafka][es] Add missing @PublicEvolving annotation on Kafka and Elasticsearch descriptor class --- .../java/org/apache/flink/table/descriptors/Elasticsearch.java | 2 ++ .../src/main/java/org/apache/flink/table/descriptors/Kafka.java | 2 ++ 2 files changed, 4 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java index fa07821e31..15614c948a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java @@ -18,6 +18,7 @@ package org.apache.flink.table.descriptors; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.MemorySize; import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; @@ -54,6 +55,7 @@ import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTO /** * Connector descriptor for the Elasticsearch search engine. */ +@PublicEvolving public class Elasticsearch extends ConnectorDescriptor { private DescriptorProperties internalProperties = new DescriptorProperties(true); diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/Kafka.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/Kafka.java index 125ece05bc..82f3e5fbf9 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/Kafka.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/Kafka.java @@ -18,6 +18,7 @@ package org.apache.flink.table.descriptors; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; @@ -50,6 +51,7 @@ import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_TYPE_V /** * Connector descriptor for the Apache Kafka message queue. */ +@PublicEvolving public class Kafka extends ConnectorDescriptor { private String version; -- Gitee From b24e4bcb4cca7e4a9d14ae3b3fff2d2678517863 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Mon, 30 Sep 2019 14:44:49 -0700 Subject: [PATCH 224/268] [FLINK-14216][table] introduce temp system functions and temp functions to FunctionCatalog adapt existing APIs to the introduction of temporary system and temp functions according to FLIP-57. This closes #9822. --- .../internal/StreamTableEnvironmentImpl.java | 6 +- .../api/internal/TableEnvironmentImpl.java | 2 +- .../flink/table/catalog/FunctionCatalog.java | 97 ++++++++++++++++--- .../internal/StreamTableEnvironmentImpl.scala | 6 +- .../plan/utils/RexNodeExtractorTest.scala | 2 +- .../table/planner/utils/TableTestBase.scala | 4 +- .../table/api/internal/TableEnvImpl.scala | 6 +- 7 files changed, 96 insertions(+), 27 deletions(-) diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java index cf2ace7c16..06b20f529a 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java @@ -147,7 +147,7 @@ public final class StreamTableEnvironmentImpl extends TableEnvironmentImpl imple public void registerFunction(String name, TableFunction tableFunction) { TypeInformation typeInfo = UserFunctionsTypeHelper.getReturnTypeOfTableFunction(tableFunction); - functionCatalog.registerTableFunction( + functionCatalog.registerTempSystemTableFunction( name, tableFunction, typeInfo @@ -160,7 +160,7 @@ public final class StreamTableEnvironmentImpl extends TableEnvironmentImpl imple TypeInformation accTypeInfo = UserFunctionsTypeHelper .getAccumulatorTypeOfAggregateFunction(aggregateFunction); - functionCatalog.registerAggregateFunction( + functionCatalog.registerTempSystemAggregateFunction( name, aggregateFunction, typeInfo, @@ -175,7 +175,7 @@ public final class StreamTableEnvironmentImpl extends TableEnvironmentImpl imple TypeInformation accTypeInfo = UserFunctionsTypeHelper .getAccumulatorTypeOfAggregateFunction(tableAggregateFunction); - functionCatalog.registerAggregateFunction( + functionCatalog.registerTempSystemAggregateFunction( name, tableAggregateFunction, typeInfo, diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java index 88aa167aa4..12e874ece3 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java @@ -163,7 +163,7 @@ public class TableEnvironmentImpl implements TableEnvironment { @Override public void registerFunction(String name, ScalarFunction function) { - functionCatalog.registerScalarFunction( + functionCatalog.registerTempSystemScalarFunction( name, function); } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java index 423b5f3300..43280b4702 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java @@ -58,9 +58,8 @@ public class FunctionCatalog implements FunctionLookup { private final CatalogManager catalogManager; - // For simplicity, currently hold registered Flink functions in memory here - // TODO: should move to catalog - private final Map userFunctions = new LinkedHashMap<>(); + private final Map tempSystemFunctions = new LinkedHashMap<>(); + private final Map tempCatalogFunctions = new LinkedHashMap<>(); /** * Temporary utility until the new type inference is fully functional. It needs to be set by the planner. @@ -75,15 +74,15 @@ public class FunctionCatalog implements FunctionLookup { this.plannerTypeInferenceUtil = plannerTypeInferenceUtil; } - public void registerScalarFunction(String name, ScalarFunction function) { + public void registerTempSystemScalarFunction(String name, ScalarFunction function) { UserFunctionsTypeHelper.validateInstantiation(function.getClass()); - registerFunction( + registerTempSystemFunction( name, new ScalarFunctionDefinition(name, function) ); } - public void registerTableFunction( + public void registerTempSystemTableFunction( String name, TableFunction function, TypeInformation resultType) { @@ -92,7 +91,7 @@ public class FunctionCatalog implements FunctionLookup { // check if class could be instantiated UserFunctionsTypeHelper.validateInstantiation(function.getClass()); - registerFunction( + registerTempSystemFunction( name, new TableFunctionDefinition( name, @@ -101,7 +100,7 @@ public class FunctionCatalog implements FunctionLookup { ); } - public void registerAggregateFunction( + public void registerTempSystemAggregateFunction( String name, UserDefinedAggregateFunction function, TypeInformation resultType, @@ -128,12 +127,71 @@ public class FunctionCatalog implements FunctionLookup { throw new TableException("Unknown function class: " + function.getClass()); } - registerFunction( + registerTempSystemFunction( name, definition ); } + public void registerTempCatalogScalarFunction(ObjectIdentifier oi, ScalarFunction function) { + UserFunctionsTypeHelper.validateInstantiation(function.getClass()); + registerTempCatalogFunction( + oi, + new ScalarFunctionDefinition(oi.getObjectName(), function) + ); + } + + public void registerTempCatalogTableFunction( + ObjectIdentifier oi, + TableFunction function, + TypeInformation resultType) { + // check if class not Scala object + UserFunctionsTypeHelper.validateNotSingleton(function.getClass()); + // check if class could be instantiated + UserFunctionsTypeHelper.validateInstantiation(function.getClass()); + + registerTempCatalogFunction( + oi, + new TableFunctionDefinition( + oi.getObjectName(), + function, + resultType) + ); + } + + public void registerTempCatalogAggregateFunction( + ObjectIdentifier oi, + UserDefinedAggregateFunction function, + TypeInformation resultType, + TypeInformation accType) { + // check if class not Scala object + UserFunctionsTypeHelper.validateNotSingleton(function.getClass()); + // check if class could be instantiated + UserFunctionsTypeHelper.validateInstantiation(function.getClass()); + + final FunctionDefinition definition; + if (function instanceof AggregateFunction) { + definition = new AggregateFunctionDefinition( + oi.getObjectName(), + (AggregateFunction) function, + resultType, + accType); + } else if (function instanceof TableAggregateFunction) { + definition = new TableAggregateFunctionDefinition( + oi.getObjectName(), + (TableAggregateFunction) function, + resultType, + accType); + } else { + throw new TableException("Unknown function class: " + function.getClass()); + } + + registerTempCatalogFunction( + oi, + definition + ); + } + public String[] getUserDefinedFunctions() { return getUserDefinedFunctionNames().toArray(new String[0]); } @@ -165,7 +223,7 @@ public class FunctionCatalog implements FunctionLookup { // Get functions registered in memory result.addAll( - userFunctions.values().stream() + tempSystemFunctions.values().stream() .map(FunctionDefinition::toString) .collect(Collectors.toSet())); @@ -204,7 +262,7 @@ public class FunctionCatalog implements FunctionLookup { } // If no corresponding function is found in catalog, check in-memory functions - userCandidate = userFunctions.get(functionName); + userCandidate = tempSystemFunctions.get(functionName); final Optional foundDefinition; if (userCandidate != null) { @@ -240,13 +298,24 @@ public class FunctionCatalog implements FunctionLookup { return plannerTypeInferenceUtil; } - private void registerFunction(String name, FunctionDefinition functionDefinition) { - // TODO: should register to catalog - userFunctions.put(normalizeName(name), functionDefinition); + private void registerTempSystemFunction(String name, FunctionDefinition functionDefinition) { + tempSystemFunctions.put(normalizeName(name), functionDefinition); + } + + private void registerTempCatalogFunction(ObjectIdentifier oi, FunctionDefinition functionDefinition) { + tempCatalogFunctions.put(normalizeObjectIdentifier(oi), functionDefinition); } @VisibleForTesting static String normalizeName(String name) { return name.toUpperCase(); } + + @VisibleForTesting + static ObjectIdentifier normalizeObjectIdentifier(ObjectIdentifier oi) { + return ObjectIdentifier.of( + oi.getCatalogName(), + oi.getDatabaseName(), + oi.getObjectName().toUpperCase()); + } } diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala index c45d324133..1384be09e5 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala @@ -137,7 +137,7 @@ class StreamTableEnvironmentImpl ( override def registerFunction[T: TypeInformation](name: String, tf: TableFunction[T]): Unit = { val typeInfo = UserFunctionsTypeHelper .getReturnTypeOfTableFunction(tf, implicitly[TypeInformation[T]]) - functionCatalog.registerTableFunction( + functionCatalog.registerTempSystemTableFunction( name, tf, typeInfo @@ -152,7 +152,7 @@ class StreamTableEnvironmentImpl ( .getReturnTypeOfAggregateFunction(f, implicitly[TypeInformation[T]]) val accTypeInfo = UserFunctionsTypeHelper .getAccumulatorTypeOfAggregateFunction(f, implicitly[TypeInformation[ACC]]) - functionCatalog.registerAggregateFunction( + functionCatalog.registerTempSystemAggregateFunction( name, f, typeInfo, @@ -168,7 +168,7 @@ class StreamTableEnvironmentImpl ( .getReturnTypeOfAggregateFunction(f, implicitly[TypeInformation[T]]) val accTypeInfo = UserFunctionsTypeHelper .getAccumulatorTypeOfAggregateFunction(f, implicitly[TypeInformation[ACC]]) - functionCatalog.registerAggregateFunction( + functionCatalog.registerTempSystemAggregateFunction( name, f, typeInfo, diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala index a49f06fa6f..f0289e4b18 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala @@ -696,7 +696,7 @@ class RexNodeExtractorTest extends RexNodeTestBase { @Test def testExtractWithUdf(): Unit = { - functionCatalog.registerScalarFunction("myUdf", Func1) + functionCatalog.registerTempSystemScalarFunction("myUdf", Func1) // amount val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) // my_udf(amount) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala index 0201d772b6..a452717e0f 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala @@ -912,7 +912,7 @@ class TestingTableEnvironment private( def registerFunction[T: TypeInformation](name: String, tf: TableFunction[T]): Unit = { val typeInfo = UserFunctionsTypeHelper .getReturnTypeOfTableFunction(tf, implicitly[TypeInformation[T]]) - functionCatalog.registerTableFunction( + functionCatalog.registerTempSystemTableFunction( name, tf, typeInfo @@ -944,7 +944,7 @@ class TestingTableEnvironment private( .getReturnTypeOfAggregateFunction(f, implicitly[TypeInformation[T]]) val accTypeInfo = UserFunctionsTypeHelper .getAccumulatorTypeOfAggregateFunction(f, implicitly[TypeInformation[ACC]]) - functionCatalog.registerAggregateFunction( + functionCatalog.registerTempSystemAggregateFunction( name, f, typeInfo, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/TableEnvImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/TableEnvImpl.scala index 0e00268879..a00ea3526b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/TableEnvImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/TableEnvImpl.scala @@ -99,7 +99,7 @@ abstract class TableEnvImpl( private def isBatchTable: Boolean = !isStreamingMode override def registerFunction(name: String, function: ScalarFunction): Unit = { - functionCatalog.registerScalarFunction( + functionCatalog.registerTempSystemScalarFunction( name, function) } @@ -117,7 +117,7 @@ abstract class TableEnvImpl( function, implicitly[TypeInformation[T]]) - functionCatalog.registerTableFunction( + functionCatalog.registerTempSystemTableFunction( name, function, resultTypeInfo) @@ -141,7 +141,7 @@ abstract class TableEnvImpl( function, implicitly[TypeInformation[ACC]]) - functionCatalog.registerAggregateFunction( + functionCatalog.registerTempSystemAggregateFunction( name, function, resultTypeInfo, -- Gitee From 2c411686d23f456cdc502abf1c6b97a61070a17d Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Wed, 16 Oct 2019 08:08:58 +0800 Subject: [PATCH 225/268] [hotfix][hbase] Put flush interval string to properties instead of parsing it in HBase descriptor This fixes the broken HBase descriptor --- .../main/java/org/apache/flink/table/descriptors/HBase.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-hbase/src/main/java/org/apache/flink/table/descriptors/HBase.java b/flink-connectors/flink-hbase/src/main/java/org/apache/flink/table/descriptors/HBase.java index 66584f9b84..9dcdff4d43 100644 --- a/flink-connectors/flink-hbase/src/main/java/org/apache/flink/table/descriptors/HBase.java +++ b/flink-connectors/flink-hbase/src/main/java/org/apache/flink/table/descriptors/HBase.java @@ -108,7 +108,7 @@ public class HBase extends ConnectorDescriptor { } /** - * Set a flush interval flushing buffered requesting if the interval passes, in milliseconds. + * Set an interval when to flushing buffered requesting if the interval passes, in milliseconds. * Defaults to not set, i.e. won't flush based on flush interval. Optional. * * @param interval flush interval. The string should be in format "{length value}{time unit label}" @@ -117,8 +117,7 @@ public class HBase extends ConnectorDescriptor { * {@link TimeUtils#parseDuration(String)}}. */ public HBase writeBufferFlushInterval(String interval) { - Duration duration = TimeUtils.parseDuration(interval); - properties.putLong(CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL, duration.toMillis()); + properties.putString(CONNECTOR_WRITE_BUFFER_FLUSH_INTERVAL, interval); return this; } -- Gitee From b1f468a82b4445a757d78184ccf4519cd0419312 Mon Sep 17 00:00:00 2001 From: Jiayi Liao Date: Wed, 16 Oct 2019 17:36:50 +0800 Subject: [PATCH 226/268] [FLINK-14355][docs] Fix compile errors in State Processor API docs --- docs/dev/libs/state_processor_api.md | 14 ++++++++------ docs/dev/libs/state_processor_api.zh.md | 14 ++++++++------ 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/docs/dev/libs/state_processor_api.md b/docs/dev/libs/state_processor_api.md index c84a13bf73..acde295720 100644 --- a/docs/dev/libs/state_processor_api.md +++ b/docs/dev/libs/state_processor_api.md @@ -241,6 +241,7 @@ public class StatefulFunctionWithTime extends KeyedProcessFunction stateDescriptor = new ValueStateDescriptor<>("state", Types.INT); state = getRuntimeContext().getState(stateDescriptor); } @@ -255,14 +256,13 @@ public class StatefulFunctionWithTime extends KeyedProcessFunction { @Override public void open(Configuration parameters) { + ValueStateDescriptor stateDescriptor = new ValueStateDescriptor<>("state", Types.INT); state = getRuntimeContext().getState(stateDescriptor); } @@ -310,9 +311,10 @@ DataSet keyedState = savepoint.readKeyedState("my-uid", new ReaderFu case class KeyedState(key: Int, value: Int) class ReaderFunction extends KeyedStateReaderFunction[Integer, KeyedState] { - var state ValueState[Integer]; + var state: ValueState[Integer]; - override def open(Configuration parameters) { + override def open(parameters: Configuration) { + val stateDescriptor = new ValueStateDescriptor("state", Types.INT); state = getRuntimeContext().getState(stateDescriptor); } diff --git a/docs/dev/libs/state_processor_api.zh.md b/docs/dev/libs/state_processor_api.zh.md index c84a13bf73..acde295720 100644 --- a/docs/dev/libs/state_processor_api.zh.md +++ b/docs/dev/libs/state_processor_api.zh.md @@ -241,6 +241,7 @@ public class StatefulFunctionWithTime extends KeyedProcessFunction stateDescriptor = new ValueStateDescriptor<>("state", Types.INT); state = getRuntimeContext().getState(stateDescriptor); } @@ -255,14 +256,13 @@ public class StatefulFunctionWithTime extends KeyedProcessFunction { @Override public void open(Configuration parameters) { + ValueStateDescriptor stateDescriptor = new ValueStateDescriptor<>("state", Types.INT); state = getRuntimeContext().getState(stateDescriptor); } @@ -310,9 +311,10 @@ DataSet keyedState = savepoint.readKeyedState("my-uid", new ReaderFu case class KeyedState(key: Int, value: Int) class ReaderFunction extends KeyedStateReaderFunction[Integer, KeyedState] { - var state ValueState[Integer]; + var state: ValueState[Integer]; - override def open(Configuration parameters) { + override def open(parameters: Configuration) { + val stateDescriptor = new ValueStateDescriptor("state", Types.INT); state = getRuntimeContext().getState(stateDescriptor); } -- Gitee From 2c50380d43e1bd0ea9989851c9a5e2ccd8e48604 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Wed, 25 Sep 2019 16:36:10 +0200 Subject: [PATCH 227/268] [FLINK-14045][runtime] Make SlotProviderStrategy class public --- .../runtime/executiongraph/SlotProviderStrategy.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SlotProviderStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SlotProviderStrategy.java index 435b0a9475..830b052220 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SlotProviderStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/SlotProviderStrategy.java @@ -35,7 +35,7 @@ import java.util.concurrent.CompletableFuture; /** * Strategy to switch between different {@link SlotProvider} allocation strategies. */ -abstract class SlotProviderStrategy { +public abstract class SlotProviderStrategy { protected final SlotProvider slotProvider; @@ -58,7 +58,7 @@ abstract class SlotProviderStrategy { * @param slotProfile profile of the requested slot * @return The future of the allocation */ - abstract CompletableFuture allocateSlot( + public abstract CompletableFuture allocateSlot( SlotRequestId slotRequestId, ScheduledUnit scheduledUnit, SlotProfile slotProfile); @@ -70,14 +70,14 @@ abstract class SlotProviderStrategy { * @param slotSharingGroupId identifying the slot request to cancel * @param cause of the cancellation */ - void cancelSlotRequest( + public void cancelSlotRequest( SlotRequestId slotRequestId, @Nullable SlotSharingGroupId slotSharingGroupId, Throwable cause) { slotProvider.cancelSlotRequest(slotRequestId, slotSharingGroupId, cause); } - static SlotProviderStrategy from( + public static SlotProviderStrategy from( ScheduleMode scheduleMode, SlotProvider slotProvider, Time allocationTimeout, -- Gitee From 6dbcddd01e86f8971b9fa87a457f36bba0d88daa Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Wed, 25 Sep 2019 16:45:59 +0200 Subject: [PATCH 228/268] [FLINK-14045][runtime] Use SlotProviderStrategy in DefaultExecutionSlotAllocator This closes #9896. --- .../DefaultExecutionSlotAllocator.java | 40 ++++++++----------- .../DefaultExecutionSlotAllocatorFactory.java | 15 ++----- .../scheduler/DefaultSchedulerFactory.java | 9 ++++- .../DefaultExecutionSlotAllocatorTest.java | 10 ++++- 4 files changed, 38 insertions(+), 36 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java index b86ff890a7..5642ab657a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java @@ -19,11 +19,11 @@ package org.apache.flink.runtime.scheduler; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.SlotProfile; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.SlotProviderStrategy; import org.apache.flink.runtime.instance.SlotSharingGroupId; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; import org.apache.flink.runtime.jobmaster.LogicalSlot; @@ -66,19 +66,15 @@ public class DefaultExecutionSlotAllocator implements ExecutionSlotAllocator { */ private final Map pendingSlotAssignments; - private final SlotProvider slotProvider; + private final SlotProviderStrategy slotProviderStrategy; private final InputsLocationsRetriever inputsLocationsRetriever; - private final Time allocationTimeout; - public DefaultExecutionSlotAllocator( - SlotProvider slotProvider, - InputsLocationsRetriever inputsLocationsRetriever, - Time allocationTimeout) { - this.slotProvider = checkNotNull(slotProvider); + SlotProviderStrategy slotProviderStrategy, + InputsLocationsRetriever inputsLocationsRetriever) { + this.slotProviderStrategy = checkNotNull(slotProviderStrategy); this.inputsLocationsRetriever = checkNotNull(inputsLocationsRetriever); - this.allocationTimeout = checkNotNull(allocationTimeout); pendingSlotAssignments = new HashMap<>(); } @@ -106,19 +102,17 @@ public class DefaultExecutionSlotAllocator implements ExecutionSlotAllocator { schedulingRequirements.getPreferredLocations(), inputsLocationsRetriever).thenCompose( (Collection preferredLocations) -> - slotProvider.allocateSlot( - slotRequestId, - new ScheduledUnit( - executionVertexId.getJobVertexId(), - slotSharingGroupId, - schedulingRequirements.getCoLocationConstraint()), - new SlotProfile( - schedulingRequirements.getResourceProfile(), - preferredLocations, - Arrays.asList(schedulingRequirements.getPreviousAllocationId()), - allPreviousAllocationIds), - true, - allocationTimeout)); + slotProviderStrategy.allocateSlot( + slotRequestId, + new ScheduledUnit( + executionVertexId.getJobVertexId(), + slotSharingGroupId, + schedulingRequirements.getCoLocationConstraint()), + new SlotProfile( + schedulingRequirements.getResourceProfile(), + preferredLocations, + Arrays.asList(schedulingRequirements.getPreviousAllocationId()), + allPreviousAllocationIds))); SlotExecutionVertexAssignment slotExecutionVertexAssignment = new SlotExecutionVertexAssignment(executionVertexId, slotFuture); @@ -129,7 +123,7 @@ public class DefaultExecutionSlotAllocator implements ExecutionSlotAllocator { (ignored, throwable) -> { pendingSlotAssignments.remove(executionVertexId); if (throwable != null) { - slotProvider.cancelSlotRequest(slotRequestId, slotSharingGroupId, throwable); + slotProviderStrategy.cancelSlotRequest(slotRequestId, slotSharingGroupId, throwable); } }); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorFactory.java index 839dc078c9..bd086930e2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorFactory.java @@ -19,8 +19,7 @@ package org.apache.flink.runtime.scheduler; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; +import org.apache.flink.runtime.executiongraph.SlotProviderStrategy; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -29,20 +28,14 @@ import static org.apache.flink.util.Preconditions.checkNotNull; */ public class DefaultExecutionSlotAllocatorFactory implements ExecutionSlotAllocatorFactory { - private final SlotProvider slotProvider; - - private final Time allocationTimeout; - - public DefaultExecutionSlotAllocatorFactory( - final SlotProvider slotProvider, - final Time allocationTimeout) { + private final SlotProviderStrategy slotProvider; + public DefaultExecutionSlotAllocatorFactory(final SlotProviderStrategy slotProvider) { this.slotProvider = checkNotNull(slotProvider); - this.allocationTimeout = checkNotNull(allocationTimeout); } @Override public ExecutionSlotAllocator createInstance(final InputsLocationsRetriever inputsLocationsRetriever) { - return new DefaultExecutionSlotAllocator(slotProvider, inputsLocationsRetriever, allocationTimeout); + return new DefaultExecutionSlotAllocator(slotProvider, inputsLocationsRetriever); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java index 0860c3c701..003035fad7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultSchedulerFactory.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobWriter; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; +import org.apache.flink.runtime.executiongraph.SlotProviderStrategy; import org.apache.flink.runtime.executiongraph.failover.flip1.RestartBackoffTimeStrategy; import org.apache.flink.runtime.executiongraph.failover.flip1.RestartBackoffTimeStrategyFactoryLoader; import org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy; @@ -77,6 +78,12 @@ public class DefaultSchedulerFactory implements SchedulerNGFactory { jobGraph.isCheckpointingEnabled()) .create(); + final SlotProviderStrategy slotProviderStrategy = SlotProviderStrategy.from( + jobGraph.getScheduleMode(), + slotProvider, + slotRequestTimeout, + true); + return new DefaultScheduler( log, jobGraph, @@ -99,7 +106,7 @@ public class DefaultSchedulerFactory implements SchedulerNGFactory { restartBackoffTimeStrategy, new DefaultExecutionVertexOperations(), new ExecutionVertexVersioner(), - new DefaultExecutionSlotAllocatorFactory(slotProvider, slotRequestTimeout)); + new DefaultExecutionSlotAllocatorFactory(slotProviderStrategy)); } private SchedulingStrategyFactory createSchedulingStrategyFactory(final ScheduleMode scheduleMode) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorTest.java index 1437b7cc3f..ce3f83711a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocatorTest.java @@ -23,8 +23,10 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotProfile; +import org.apache.flink.runtime.executiongraph.SlotProviderStrategy; import org.apache.flink.runtime.instance.SlotSharingGroupId; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; @@ -278,7 +280,13 @@ public class DefaultExecutionSlotAllocatorTest extends TestLogger { } private DefaultExecutionSlotAllocator createExecutionSlotAllocator(InputsLocationsRetriever inputsLocationsRetriever) { - return new DefaultExecutionSlotAllocator(slotProvider, inputsLocationsRetriever, Time.seconds(10)); + return new DefaultExecutionSlotAllocator( + SlotProviderStrategy.from( + ScheduleMode.EAGER, + slotProvider, + Time.seconds(10), + true), + inputsLocationsRetriever); } private List createSchedulingRequirements(ExecutionVertexID... executionVertexIds) { -- Gitee From 88b48619e2734505a6c2ba0d53168528bc0dc143 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Mon, 14 Oct 2019 11:55:25 +0200 Subject: [PATCH 229/268] [hotfix][runtime] Replace Arrays.asList() with Collections.singletonList() --- .../flink/runtime/scheduler/DefaultExecutionSlotAllocator.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java index 5642ab657a..22599a632d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionSlotAllocator.java @@ -36,7 +36,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -111,7 +110,7 @@ public class DefaultExecutionSlotAllocator implements ExecutionSlotAllocator { new SlotProfile( schedulingRequirements.getResourceProfile(), preferredLocations, - Arrays.asList(schedulingRequirements.getPreviousAllocationId()), + Collections.singletonList(schedulingRequirements.getPreviousAllocationId()), allPreviousAllocationIds))); SlotExecutionVertexAssignment slotExecutionVertexAssignment = -- Gitee From 46f8b592c84b8406aab5307f20e779e55d45de80 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 15 Oct 2019 10:55:07 +0800 Subject: [PATCH 230/268] [FLINK-14365][tests] Annotate MiniCluster tests in core modules with AlsoRunWithSchedulerNG This closes #9901. --- .../test/java/org/apache/flink/client/program/ClientTest.java | 3 +++ .../org/apache/flink/runtime/webmonitor/WebFrontendITCase.java | 3 +++ .../flink/runtime/webmonitor/handlers/JarRunHandlerTest.java | 3 +++ .../flink/runtime/webmonitor/history/HistoryServerTest.java | 3 +++ .../scala/org/apache/flink/api/scala/ScalaShellITCase.scala | 3 +++ 5 files changed, 15 insertions(+) diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java index 3b54b8efd5..b31f95cc93 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java @@ -39,12 +39,14 @@ import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.testutils.MiniClusterResource; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.NetUtils; import org.apache.flink.util.TestLogger; import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -59,6 +61,7 @@ import static org.mockito.Mockito.mock; /** * Simple and maybe stupid test to check the {@link ClusterClient} class. */ +@Category(AlsoRunWithSchedulerNG.class) public class ClientTest extends TestLogger { @ClassRule diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java index 21d4a85923..1202499309 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.runtime.webmonitor.testutils.HttpTestClient; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; @@ -46,6 +47,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.io.File; import java.io.InputStream; @@ -68,6 +70,7 @@ import static org.junit.Assert.fail; /** * Tests for the WebFrontend. */ +@Category(AlsoRunWithSchedulerNG.class) public class WebFrontendITCase extends TestLogger { private static final int NUM_TASK_MANAGERS = 2; 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 b4182c735c..1b306300c6 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 @@ -28,11 +28,13 @@ import org.apache.flink.runtime.rest.util.RestClientException; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.MiniClusterResource; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import java.nio.file.Files; @@ -46,6 +48,7 @@ import static org.junit.Assert.assertTrue; /** * Tests for the {@link JarRunHandler}. */ +@Category(AlsoRunWithSchedulerNG.class) public class JarRunHandlerTest extends TestLogger { @ClassRule diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java index fa932dc266..7511300037 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonFactory; @@ -42,6 +43,7 @@ import org.junit.Assert; import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -63,6 +65,7 @@ import java.util.concurrent.TimeUnit; * Tests for the HistoryServer. */ @RunWith(Parameterized.class) +@Category(AlsoRunWithSchedulerNG.class) public class HistoryServerTest extends TestLogger { @ClassRule 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 8fd94e5d69..3c158a818f 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 @@ -25,8 +25,10 @@ import org.apache.flink.runtime.clusterframework.BootstrapTools import org.apache.flink.runtime.minicluster.MiniCluster import org.apache.flink.runtime.testutils.{MiniClusterResource, MiniClusterResourceConfiguration} import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG import org.apache.flink.util.TestLogger import org.junit._ +import org.junit.experimental.categories.Category import org.junit.rules.TemporaryFolder import scala.tools.nsc.Settings @@ -441,6 +443,7 @@ class ScalaShellITCase extends TestLogger { } +@Category(Array(classOf[AlsoRunWithSchedulerNG])) object ScalaShellITCase { val configuration = new Configuration() -- Gitee From d632018ed915a8ca7c5f7b180e023a87cfae9a21 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 11 Sep 2019 17:36:23 +0200 Subject: [PATCH 231/268] [FLINK-14290] Add Pipeline as a common base class of DataSet and DataStream plans For now, only DataSet Plan implements this, as a follow-up, StreamGraph should also implement this. --- .../org/apache/flink/api/common/Plan.java | 3 +- .../org/apache/flink/api/dag/Pipeline.java | 28 +++++++++++++++++++ 2 files changed, 30 insertions(+), 1 deletion(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/dag/Pipeline.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 0937842fea..0e71bcda97 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 @@ -22,6 +22,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry; import org.apache.flink.api.common.operators.GenericDataSinkBase; import org.apache.flink.api.common.operators.Operator; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.util.Visitable; import org.apache.flink.util.Visitor; @@ -46,7 +47,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; * operators of the data flow can be reached via backwards traversal

    . */ @Internal -public class Plan implements Visitable> { +public class Plan implements Visitable>, Pipeline { /** * A collection of all sinks in the plan. Since the plan is traversed from the sinks to the diff --git a/flink-core/src/main/java/org/apache/flink/api/dag/Pipeline.java b/flink-core/src/main/java/org/apache/flink/api/dag/Pipeline.java new file mode 100644 index 0000000000..ab75e03671 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/dag/Pipeline.java @@ -0,0 +1,28 @@ +/* + * 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.dag; + +import org.apache.flink.annotation.Internal; + +/** + * Common interface for Flink pipelines. + */ +@Internal +public interface Pipeline {} -- Gitee From 8092dafa05f1dea3cbb63888649362ca62573f24 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 11 Sep 2019 17:37:33 +0200 Subject: [PATCH 232/268] [FLINK-14290] Add Pipeline translation utils for getting a JobGraph from a FlinkPipeline For now, we can only translate DataSet Plans, in the future we also need this for DataStream StreamGraps. --- .../client/FlinkPipelineTranslationUtil.java | 52 ++++++++++++ .../flink/client/FlinkPipelineTranslator.java | 42 ++++++++++ .../apache/flink/client/PlanTranslator.java | 82 +++++++++++++++++++ 3 files changed, 176 insertions(+) create mode 100644 flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java create mode 100644 flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslator.java create mode 100644 flink-clients/src/main/java/org/apache/flink/client/PlanTranslator.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java new file mode 100644 index 0000000000..61bda14a6d --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.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.client; + +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.JobGraph; + +/** + * Utility for transforming {@link Pipeline FlinkPipelines} into a {@link JobGraph}. This uses + * reflection or service discovery to find the right {@link FlinkPipelineTranslator} for a given + * subclass of {@link Pipeline}. + */ +public final class FlinkPipelineTranslationUtil { + + /** + * Transmogrifies the given {@link Pipeline} to a {@link JobGraph}. + */ + public static JobGraph getJobGraph( + Pipeline pipeline, + Configuration optimizerConfiguration, + int defaultParallelism) { + + PlanTranslator planToJobGraphTransmogrifier = new PlanTranslator(); + + if (planToJobGraphTransmogrifier.canTranslate(pipeline)) { + // we have a DataSet program + return planToJobGraphTransmogrifier.translate(pipeline, + optimizerConfiguration, + defaultParallelism); + } + + throw new RuntimeException("Cannot find transmogrifier for given pipeline: " + pipeline); + } +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslator.java b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslator.java new file mode 100644 index 0000000000..c224422599 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslator.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.flink.client; + +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.JobGraph; + +/** + * This can be used to turn a {@link Pipeline} into a {@link JobGraph}. There will be + * implementations for the different pipeline APIs that Flink supports. + */ +public interface FlinkPipelineTranslator { + + /** + * Creates a {@link JobGraph} from the given {@link Pipeline} and attaches the given jar + * files and classpaths to the {@link JobGraph}. + */ + JobGraph translate( + Pipeline pipeline, + Configuration optimizerConfiguration, + int defaultParallelism); + + boolean canTranslate(Pipeline pipeline); +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/PlanTranslator.java b/flink-clients/src/main/java/org/apache/flink/client/PlanTranslator.java new file mode 100644 index 0000000000..ae18b6b24a --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/PlanTranslator.java @@ -0,0 +1,82 @@ +/* + * 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.client; + +import org.apache.flink.api.common.Plan; +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.optimizer.DataStatistics; +import org.apache.flink.optimizer.Optimizer; +import org.apache.flink.optimizer.plan.OptimizedPlan; +import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; +import org.apache.flink.runtime.jobgraph.JobGraph; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * {@link FlinkPipelineTranslator} for DataSet API {@link Plan Plans}. + */ +public class PlanTranslator implements FlinkPipelineTranslator { + + private static final Logger LOG = LoggerFactory.getLogger(PlanTranslator.class); + + @Override + public JobGraph translate( + Pipeline pipeline, + Configuration optimizerConfiguration, + int defaultParallelism) { + checkArgument(pipeline instanceof Plan, "Given pipeline is not a DataSet Plan."); + + Plan plan = (Plan) pipeline; + setDefaultParallelism(plan, defaultParallelism); + return compilePlan(plan, optimizerConfiguration); + } + + private void setDefaultParallelism(Plan plan, int defaultParallelism) { + if (defaultParallelism > 0 && plan.getDefaultParallelism() <= 0) { + LOG.debug( + "Changing plan default parallelism from {} to {}", + plan.getDefaultParallelism(), + defaultParallelism); + plan.setDefaultParallelism(defaultParallelism); + } + + LOG.debug( + "Set parallelism {}, plan default parallelism {}", + defaultParallelism, + plan.getDefaultParallelism()); + } + + private JobGraph compilePlan(Plan plan, Configuration optimizerConfiguration) { + Optimizer optimizer = new Optimizer(new DataStatistics(), optimizerConfiguration); + OptimizedPlan optimizedPlan = optimizer.compile(plan); + + JobGraphGenerator jobGraphGenerator = new JobGraphGenerator(optimizerConfiguration); + return jobGraphGenerator.compileJobGraph(optimizedPlan, plan.getJobId()); + } + + @Override + public boolean canTranslate(Pipeline pipeline) { + return pipeline instanceof Plan; + } +} -- Gitee From fa872ebfe47bc46c0afba19091caf84f2c29a7b9 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 11 Sep 2019 17:38:31 +0200 Subject: [PATCH 233/268] [FLINK-14290] Change DataSet PlanExecutors to use the new pipeline translation util This decouples JobGraph generation from the executors and environments. --- .../org/apache/flink/client/ClientUtils.java | 16 ++++ .../apache/flink/client/LocalExecutor.java | 96 +++++++++---------- .../apache/flink/client/RemoteExecutor.java | 34 ++++--- .../client/program/ContextEnvironment.java | 30 +++--- .../apache/flink/api/common/PlanExecutor.java | 3 +- .../apache/flink/api/java/FlinkILoopTest.java | 4 +- 6 files changed, 106 insertions(+), 77 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java index ee03705787..9fb4ce5b9f 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java +++ b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java @@ -18,7 +18,9 @@ package org.apache.flink.client; +import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; +import org.apache.flink.runtime.jobgraph.JobGraph; import java.io.File; import java.io.IOException; @@ -33,6 +35,20 @@ import java.util.jar.JarFile; public enum ClientUtils { ; + /** + * Adds the given jar files to the {@link JobGraph} via {@link JobGraph#addJar}. This will + * throw an exception if a jar URL is not valid. + */ + public static void addJarFiles(JobGraph jobGraph, List jarFilesToAttach) { + for (URL jar : jarFilesToAttach) { + try { + jobGraph.addJar(new Path(jar.toURI())); + } catch (URISyntaxException e) { + throw new RuntimeException("URL is invalid. This should not happen.", e); + } + } + } + public static void checkJarFile(URL jar) throws IOException { File jarFile; try { diff --git a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java index d9444d5e8a..290cff9d1c 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java @@ -21,26 +21,25 @@ package org.apache.flink.client; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.PlanExecutor; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.minicluster.JobExecutorService; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import java.util.Collections; + import static org.apache.flink.util.Preconditions.checkNotNull; /** * A PlanExecutor that runs Flink programs on a local embedded Flink runtime instance. * - *

    By simply calling the {@link #executePlan(org.apache.flink.api.common.Plan)} method, + *

    By simply calling the {@link #executePlan(Pipeline)} method, * this executor still start up and shut down again immediately after the program finished.

    * *

    To use this executor to execute many dataflow programs that constitute one job together, @@ -59,22 +58,29 @@ public class LocalExecutor extends PlanExecutor { this.baseConfiguration = checkNotNull(conf); } - private JobExecutorService createJobExecutorService(Configuration configuration) throws Exception { + private JobExecutorService createJobExecutorService( + JobGraph jobGraph, Configuration configuration) throws Exception { if (!configuration.contains(RestOptions.BIND_PORT)) { configuration.setString(RestOptions.BIND_PORT, "0"); } - final MiniClusterConfiguration miniClusterConfiguration = new MiniClusterConfiguration.Builder() - .setConfiguration(configuration) - .setNumTaskManagers( - configuration.getInteger( - ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, - ConfigConstants.DEFAULT_LOCAL_NUMBER_TASK_MANAGER)) - .setRpcServiceSharing(RpcServiceSharing.SHARED) - .setNumSlotsPerTaskManager( - configuration.getInteger( - TaskManagerOptions.NUM_TASK_SLOTS, 1)) - .build(); + int numTaskManagers = configuration.getInteger( + ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, + ConfigConstants.DEFAULT_LOCAL_NUMBER_TASK_MANAGER); + + // we have to use the maximum parallelism as a default here, otherwise streaming + // pipelines would not run + int numSlotsPerTaskManager = configuration.getInteger( + TaskManagerOptions.NUM_TASK_SLOTS, + jobGraph.getMaximumParallelism()); + + final MiniClusterConfiguration miniClusterConfiguration = + new MiniClusterConfiguration.Builder() + .setConfiguration(configuration) + .setNumTaskManagers(numTaskManagers) + .setRpcServiceSharing(RpcServiceSharing.SHARED) + .setNumSlotsPerTaskManager(numSlotsPerTaskManager) + .build(); final MiniCluster miniCluster = new MiniCluster(miniClusterConfiguration); miniCluster.start(); @@ -91,50 +97,38 @@ public class LocalExecutor extends PlanExecutor { * after the job finished. If the job runs in session mode, the executor is kept alive until * no more references to the executor exist.

    * - * @param plan The plan of the program to execute. + * @param pipeline The pipeline of the program to execute. * @return The net runtime of the program, in milliseconds. * * @throws Exception Thrown, if either the startup of the local execution context, or the execution * caused an exception. */ @Override - public JobExecutionResult executePlan(Plan plan) throws Exception { - checkNotNull(plan); - - final Configuration jobExecutorServiceConfiguration = configureExecution(plan); - - try (final JobExecutorService executorService = createJobExecutorService(jobExecutorServiceConfiguration)) { + public JobExecutionResult executePlan(Pipeline pipeline) throws Exception { + checkNotNull(pipeline); + + // This is a quirk in how LocalEnvironment used to work. It sets the default parallelism + // to * . Might be questionable but we keep the behaviour + // for now. + if (pipeline instanceof Plan) { + Plan plan = (Plan) pipeline; + final int slotsPerTaskManager = baseConfiguration.getInteger( + TaskManagerOptions.NUM_TASK_SLOTS, plan.getMaximumParallelism()); + final int numTaskManagers = baseConfiguration.getInteger( + ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); + + plan.setDefaultParallelism(slotsPerTaskManager * numTaskManagers); + } - Optimizer pc = new Optimizer(new DataStatistics(), jobExecutorServiceConfiguration); - OptimizedPlan op = pc.compile(plan); + JobGraph jobGraph = FlinkPipelineTranslationUtil.getJobGraph(pipeline, + baseConfiguration, + 1); - JobGraphGenerator jgg = new JobGraphGenerator(jobExecutorServiceConfiguration); - JobGraph jobGraph = jgg.compileJobGraph(op, plan.getJobId()); + jobGraph.setAllowQueuedScheduling(true); + try (final JobExecutorService executorService = createJobExecutorService(jobGraph, + baseConfiguration)) { return executorService.executeJobBlocking(jobGraph); } } - - private Configuration configureExecution(final Plan plan) { - final Configuration executorConfiguration = createExecutorServiceConfig(plan); - setPlanParallelism(plan, executorConfiguration); - return executorConfiguration; - } - - private Configuration createExecutorServiceConfig(final Plan plan) { - final Configuration newConfiguration = new Configuration(); - newConfiguration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, plan.getMaximumParallelism()); - newConfiguration.addAll(baseConfiguration); - return newConfiguration; - } - - private void setPlanParallelism(final Plan plan, final Configuration executorServiceConfig) { - // TODO: Set job's default parallelism to max number of slots - final int slotsPerTaskManager = executorServiceConfig.getInteger( - TaskManagerOptions.NUM_TASK_SLOTS, plan.getMaximumParallelism()); - final int numTaskManagers = executorServiceConfig.getInteger( - ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); - - plan.setDefaultParallelism(slotsPerTaskManager * numTaskManagers); - } } diff --git a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java index 4d0dfd9c7d..d78a04fd3c 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java @@ -19,14 +19,14 @@ package org.apache.flink.client; import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.PlanExecutor; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.RestOptions; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.jobgraph.JobGraph; import java.net.InetSocketAddress; import java.net.URL; @@ -114,19 +114,31 @@ public class RemoteExecutor extends PlanExecutor { // ------------------------------------------------------------------------ @Override - public JobExecutionResult executePlan(Plan plan) throws Exception { + public JobExecutionResult executePlan(Pipeline plan) throws Exception { checkNotNull(plan); - try (ClusterClient client = new RestClusterClient<>(clientConfiguration, "RemoteExecutor")) { - ClassLoader classLoader = ClientUtils.buildUserCodeClassLoader(jarFiles, globalClasspaths, getClass().getClassLoader()); - - return client.run( + JobGraph jobGraph = FlinkPipelineTranslationUtil.getJobGraph( plan, + clientConfiguration, + getDefaultParallelism()); + + ClientUtils.addJarFiles(jobGraph, jarFiles); + jobGraph.setClasspaths(globalClasspaths); + + ClassLoader userCodeClassLoader = ClientUtils.buildUserCodeClassLoader( jarFiles, - globalClasspaths, - classLoader, - defaultParallelism, - SavepointRestoreSettings.none()).getJobExecutionResult(); + this.globalClasspaths, + getClass().getClassLoader()); + + return executePlanWithJars(jobGraph, userCodeClassLoader); + } + + private JobExecutionResult executePlanWithJars(JobGraph jobGraph, ClassLoader classLoader) throws Exception { + checkNotNull(jobGraph); + checkNotNull(classLoader); + + try (ClusterClient client = new RestClusterClient<>(clientConfiguration, "RemoteExecutor")) { + return client.submitJob(jobGraph, classLoader).getJobExecutionResult(); } } } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java index 4d46595512..0ceb850681 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java @@ -21,9 +21,11 @@ package org.apache.flink.client.program; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.client.ClientUtils; +import org.apache.flink.client.FlinkPipelineTranslationUtil; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import java.net.URL; @@ -64,17 +66,21 @@ public class ContextEnvironment extends ExecutionEnvironment { public JobExecutionResult execute(String jobName) throws Exception { verifyExecuteIsCalledOnceWhenInDetachedMode(); - final Plan plan = createProgramPlan(jobName); - final JobSubmissionResult jobSubmissionResult = client.run( - plan, - jarFilesToAttach, - classpathsToAttach, - userCodeClassLoader, - getParallelism(), - savepointSettings); - - lastJobExecutionResult = jobSubmissionResult.getJobExecutionResult(); - return lastJobExecutionResult; + Plan plan = createProgramPlan(jobName); + + JobGraph jobGraph = FlinkPipelineTranslationUtil.getJobGraph( + plan, + client.getFlinkConfiguration(), + getParallelism()); + + ClientUtils.addJarFiles(jobGraph, this.jarFilesToAttach); + jobGraph.setClasspaths(this.classpathsToAttach); + + this.lastJobExecutionResult = client + .submitJob(jobGraph, this.userCodeClassLoader) + .getJobExecutionResult(); + + return this.lastJobExecutionResult; } private void verifyExecuteIsCalledOnceWhenInDetachedMode() { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java b/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java index 2095c63c45..83a8e17113 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java @@ -19,6 +19,7 @@ package org.apache.flink.api.common; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.configuration.Configuration; import java.net.URL; @@ -62,7 +63,7 @@ public abstract class PlanExecutor { * * @throws Exception Thrown, if job submission caused an exception. */ - public abstract JobExecutionResult executePlan(Plan plan) throws Exception; + public abstract JobExecutionResult executePlan(Pipeline plan) throws Exception; // ------------------------------------------------------------------------ // Executor Factories diff --git a/flink-scala-shell/src/test/java/org/apache/flink/api/java/FlinkILoopTest.java b/flink-scala-shell/src/test/java/org/apache/flink/api/java/FlinkILoopTest.java index eddf66c6d3..5b46c54093 100644 --- a/flink-scala-shell/src/test/java/org/apache/flink/api/java/FlinkILoopTest.java +++ b/flink-scala-shell/src/test/java/org/apache/flink/api/java/FlinkILoopTest.java @@ -19,8 +19,8 @@ package org.apache.flink.api.java; import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.PlanExecutor; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.api.scala.FlinkILoop; import org.apache.flink.configuration.Configuration; @@ -128,7 +128,7 @@ public class FlinkILoopTest extends TestLogger { private List globalClasspaths; @Override - public JobExecutionResult executePlan(Plan plan) throws Exception { + public JobExecutionResult executePlan(Pipeline plan) throws Exception { return null; } -- Gitee From 55eb4b254eb531287ec84c63874b479d42f8e315 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 12 Sep 2019 13:37:18 +0200 Subject: [PATCH 234/268] [FLINK-14290] Add support for StreamGraph to pipeline translation util --- .../client/FlinkPipelineTranslationUtil.java | 38 +++++++++++- .../streaming/api/graph/StreamGraph.java | 3 +- .../api/graph/StreamGraphTranslator.java | 59 +++++++++++++++++++ 3 files changed, 98 insertions(+), 2 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphTranslator.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java index 61bda14a6d..33c8027708 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java +++ b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java @@ -47,6 +47,42 @@ public final class FlinkPipelineTranslationUtil { defaultParallelism); } - throw new RuntimeException("Cannot find transmogrifier for given pipeline: " + pipeline); + FlinkPipelineTranslator streamGraphTranslator = reflectStreamGraphTranslator(); + + if (!streamGraphTranslator.canTranslate(pipeline)) { + throw new RuntimeException("Translator " + streamGraphTranslator + " cannot translate " + + "the given pipeline " + pipeline + "."); + } + + return streamGraphTranslator.translate(pipeline, + optimizerConfiguration, + defaultParallelism); + } + + private static FlinkPipelineTranslator reflectStreamGraphTranslator() { + // Try our luck with StreamGraph translation. We have to load a StreamGraphTranslator + // via reflection because the dependencies of flink-streaming-java are inverted compared + // to flink-java. For flink-java does not depend on runtime, clients or optimizer and + // we have the translation code in clients/optimizer. On the other hand, + // flink-streaming-java depends on runtime and clients. + + Class streamGraphTranslatorClass; + try { + streamGraphTranslatorClass = Class.forName( + "org.apache.flink.streaming.api.graph.StreamGraphTranslator", + true, + FlinkPipelineTranslationUtil.class.getClassLoader()); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Could not load StreamGraphTranslator.", e); + } + + FlinkPipelineTranslator streamGraphTranslator; + try { + streamGraphTranslator = + (FlinkPipelineTranslator) streamGraphTranslatorClass.newInstance(); + } catch (InstantiationException | IllegalAccessException e) { + throw new RuntimeException("Could not instantiate StreamGraphTranslator.", e); + } + return streamGraphTranslator; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java index f4741fb315..522c6fc9ff 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java @@ -26,6 +26,7 @@ import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.common.operators.ResourceSpec; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; @@ -71,7 +72,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; * */ @Internal -public class StreamGraph extends StreamingPlan { +public class StreamGraph extends StreamingPlan implements Pipeline { private static final Logger LOG = LoggerFactory.getLogger(StreamGraph.class); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphTranslator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphTranslator.java new file mode 100644 index 0000000000..ebd554fc9c --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphTranslator.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.streaming.api.graph; + +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.client.FlinkPipelineTranslator; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.JobGraph; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * {@link FlinkPipelineTranslator} for DataStream API {@link StreamGraph StreamGraphs}. + * + *

    Note: this is used through reflection in + * {@link org.apache.flink.client.FlinkPipelineTranslationUtil}. + */ +@SuppressWarnings("unused") +public class StreamGraphTranslator implements FlinkPipelineTranslator { + + private static final Logger LOG = LoggerFactory.getLogger(StreamGraphTranslator.class); + + @Override + public JobGraph translate( + Pipeline pipeline, + Configuration optimizerConfiguration, + int defaultParallelism) { + checkArgument(pipeline instanceof StreamGraph, + "Given pipeline is not a DataStream StreamGraph."); + + StreamGraph streamGraph = (StreamGraph) pipeline; + return streamGraph.getJobGraph(); + } + + @Override + public boolean canTranslate(Pipeline pipeline) { + return pipeline instanceof StreamGraph; + } +} -- Gitee From b47c13c00be47d3a9132761134261d80295920b4 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 12 Sep 2019 13:37:39 +0200 Subject: [PATCH 235/268] [FLINK-14290] Use LocalExecutor in LocalStreamEnvironment --- .../environment/LocalStreamEnvironment.java | 48 ++----------------- 1 file changed, 4 insertions(+), 44 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java index f655af62f0..54f4354591 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java @@ -20,18 +20,11 @@ package org.apache.flink.streaming.api.environment; import org.apache.flink.annotation.Public; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.PlanExecutor; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.RestOptions; -import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.minicluster.MiniCluster; -import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; import org.apache.flink.streaming.api.graph.StreamGraph; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import javax.annotation.Nonnull; /** @@ -45,8 +38,6 @@ import javax.annotation.Nonnull; @Public public class LocalStreamEnvironment extends StreamExecutionEnvironment { - private static final Logger LOG = LoggerFactory.getLogger(LocalStreamEnvironment.class); - private final Configuration configuration; /** @@ -83,42 +74,11 @@ public class LocalStreamEnvironment extends StreamExecutionEnvironment { */ @Override public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { - JobGraph jobGraph = streamGraph.getJobGraph(); - jobGraph.setAllowQueuedScheduling(true); - - Configuration configuration = new Configuration(); - configuration.addAll(jobGraph.getJobConfiguration()); - configuration.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "0"); - - // add (and override) the settings with what the user defined - configuration.addAll(this.configuration); - - if (!configuration.contains(RestOptions.BIND_PORT)) { - configuration.setString(RestOptions.BIND_PORT, "0"); - } - - int numSlotsPerTaskManager = configuration.getInteger(TaskManagerOptions.NUM_TASK_SLOTS, jobGraph.getMaximumParallelism()); - - MiniClusterConfiguration cfg = new MiniClusterConfiguration.Builder() - .setConfiguration(configuration) - .setNumSlotsPerTaskManager(numSlotsPerTaskManager) - .build(); - - if (LOG.isInfoEnabled()) { - LOG.info("Running job on local embedded Flink mini cluster"); - } - - MiniCluster miniCluster = new MiniCluster(cfg); - try { - miniCluster.start(); - configuration.setInteger(RestOptions.PORT, miniCluster.getRestAddress().get().getPort()); - - return miniCluster.executeJobBlocking(jobGraph); - } - finally { + final PlanExecutor executor = PlanExecutor.createLocalExecutor(configuration); + return executor.executePlan(streamGraph); + } finally { transformations.clear(); - miniCluster.close(); } } } -- Gitee From 14b19dd4ac9845616604bfc72de6e84506e56caa Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 19 Sep 2019 16:37:33 +0200 Subject: [PATCH 236/268] [FLINK-14290] Add SavepointRestoreSettings to StreamGraph and Generators We need this to be able to set SavepointRestoreSettings on a StreamGraph that we want to execute via an Executor (PlanExecutor). If we don't set the settings on the StreamGraph we would have to pass them to the Executor, as we now pass them to the ClusterClient. This can make the Executor/ClusterClient unaware of this streaming-only setting. --- .../streaming/api/graph/StreamGraph.java | 13 +++++++++++- .../api/graph/StreamGraphGenerator.java | 9 ++++++++- .../api/graph/StreamingJobGraphGenerator.java | 2 ++ .../api/graph/StreamGraphGeneratorTest.java | 18 +++++++++++++++++ .../graph/StreamingJobGraphGeneratorTest.java | 20 ++++++++++++++++++- 5 files changed, 59 insertions(+), 3 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java index 522c6fc9ff..ec6992fd7a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java @@ -33,6 +33,7 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.MissingTypeInfo; import org.apache.flink.optimizer.plan.StreamingPlan; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.state.StateBackend; @@ -84,6 +85,7 @@ public class StreamGraph extends StreamingPlan implements Pipeline { private final ExecutionConfig executionConfig; private final CheckpointConfig checkpointConfig; + private SavepointRestoreSettings savepointRestoreSettings = SavepointRestoreSettings.none(); private ScheduleMode scheduleMode; @@ -111,9 +113,10 @@ public class StreamGraph extends StreamingPlan implements Pipeline { private StateBackend stateBackend; private Set> iterationSourceSinkPairs; - public StreamGraph(ExecutionConfig executionConfig, CheckpointConfig checkpointConfig) { + public StreamGraph(ExecutionConfig executionConfig, CheckpointConfig checkpointConfig, SavepointRestoreSettings savepointRestoreSettings) { this.executionConfig = checkNotNull(executionConfig); this.checkpointConfig = checkNotNull(checkpointConfig); + this.savepointRestoreSettings = checkNotNull(savepointRestoreSettings); // create an empty new stream graph. clear(); @@ -142,6 +145,14 @@ public class StreamGraph extends StreamingPlan implements Pipeline { return checkpointConfig; } + public void setSavepointRestoreSettings(SavepointRestoreSettings savepointRestoreSettings) { + this.savepointRestoreSettings = savepointRestoreSettings; + } + + public SavepointRestoreSettings getSavepointRestoreSettings() { + return savepointRestoreSettings; + } + public String getJobName() { return jobName; } 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 bd6a4b54b3..9a4eafd967 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 @@ -24,6 +24,7 @@ import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.runtime.state.StateBackend; @@ -107,6 +108,8 @@ public class StreamGraphGenerator { private final CheckpointConfig checkpointConfig; + private SavepointRestoreSettings savepointRestoreSettings = SavepointRestoreSettings.none(); + private StateBackend stateBackend; private boolean chaining = true; @@ -193,8 +196,12 @@ public class StreamGraphGenerator { return this; } + public void setSavepointRestoreSettings(SavepointRestoreSettings savepointRestoreSettings) { + this.savepointRestoreSettings = savepointRestoreSettings; + } + public StreamGraph generate() { - streamGraph = new StreamGraph(executionConfig, checkpointConfig); + streamGraph = new StreamGraph(executionConfig, checkpointConfig, savepointRestoreSettings); streamGraph.setStateBackend(stateBackend); streamGraph.setChaining(chaining); streamGraph.setScheduleMode(scheduleMode); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index 58b4989dd7..83a8103152 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -166,6 +166,8 @@ public class StreamingJobGraphGenerator { configureCheckpointing(); + jobGraph.setSavepointRestoreSettings(streamGraph.getSavepointRestoreSettings()); + JobGraphGenerator.addUserArtifactEntries(streamGraph.getUserArtifacts(), jobGraph); // set the ExecutionConfig last when it has been finalized diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java index e09334aa8c..fc057e1027 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java @@ -23,10 +23,12 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.streaming.api.datastream.ConnectedStreams; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.IterativeStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.co.CoMapFunction; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; @@ -53,11 +55,14 @@ import org.junit.Test; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.List; +import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -68,6 +73,19 @@ import static org.junit.Assert.assertTrue; @SuppressWarnings("serial") public class StreamGraphGeneratorTest { + @Test + public void generatorForwardsSavepointRestoreSettings() { + StreamGraphGenerator streamGraphGenerator = + new StreamGraphGenerator(Collections.emptyList(), + new ExecutionConfig(), + new CheckpointConfig()); + + streamGraphGenerator.setSavepointRestoreSettings(SavepointRestoreSettings.forPath("hello")); + + StreamGraph streamGraph = streamGraphGenerator.generate(); + assertThat(streamGraph.getSavepointRestoreSettings().getRestorePath(), is("hello")); + } + @Test public void testBufferTimeout() { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java index 1334d2830d..18b25f7936 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.api.graph; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.MapFunction; @@ -37,6 +38,7 @@ import org.apache.flink.runtime.jobgraph.InputOutputFormatVertex; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; @@ -47,6 +49,7 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.datastream.IterativeStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; @@ -68,10 +71,12 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import static org.hamcrest.CoreMatchers.is; 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; /** @@ -137,7 +142,7 @@ public class StreamingJobGraphGeneratorTest extends TestLogger { @Test public void testDisabledCheckpointing() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - StreamGraph streamGraph = new StreamGraph(env.getConfig(), env.getCheckpointConfig()); + StreamGraph streamGraph = new StreamGraph(env.getConfig(), env.getCheckpointConfig(), SavepointRestoreSettings.none()); assertFalse("Checkpointing enabled", streamGraph.getCheckpointConfig().isCheckpointingEnabled()); JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(streamGraph); @@ -146,6 +151,19 @@ public class StreamingJobGraphGeneratorTest extends TestLogger { assertEquals(Long.MAX_VALUE, snapshottingSettings.getCheckpointCoordinatorConfiguration().getCheckpointInterval()); } + @Test + public void generatorForwardsSavepointRestoreSettings() { + StreamGraph streamGraph = new StreamGraph( + new ExecutionConfig(), + new CheckpointConfig(), + SavepointRestoreSettings.forPath("hello")); + + JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(streamGraph); + + SavepointRestoreSettings savepointRestoreSettings = jobGraph.getSavepointRestoreSettings(); + assertThat(savepointRestoreSettings.getRestorePath(), is("hello")); + } + /** * Verifies that the chain start/end is correctly set. */ -- Gitee From 85e5a77de130f4b064d3c70671f1fd7f3fc046e2 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 19 Sep 2019 16:42:19 +0200 Subject: [PATCH 237/268] [FLINK-14290] Use RemoteExecutor in RemoteStreamEnvironment --- .../environment/RemoteStreamEnvironment.java | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java index 13bde9662e..b393e5ee3d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.PlanExecutor; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; @@ -225,7 +226,6 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment { ) throws ProgramInvocationException { StreamGraph streamGraph = streamExecutionEnvironment.getStreamGraph(jobName); return executeRemotely(streamGraph, - streamExecutionEnvironment.getClass().getClassLoader(), streamExecutionEnvironment.getConfig(), jarFiles, host, @@ -242,7 +242,6 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment { * @throws ProgramInvocationException */ private static JobExecutionResult executeRemotely(StreamGraph streamGraph, - ClassLoader envClassLoader, ExecutionConfig executionConfig, List jarFiles, String host, @@ -255,8 +254,6 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment { LOG.info("Running remotely at {}:{}", host, port); } - ClassLoader userCodeClassLoader = ClientUtils.buildUserCodeClassLoader(jarFiles, globalClasspaths, envClassLoader); - Configuration configuration = new Configuration(); configuration.addAll(clientConfiguration); @@ -274,13 +271,18 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment { streamGraph.getJobGraph().getJobID(), e); } - if (savepointRestoreSettings == null) { - savepointRestoreSettings = SavepointRestoreSettings.none(); + if (savepointRestoreSettings != null) { + streamGraph.setSavepointRestoreSettings(savepointRestoreSettings); } try { - return client.run(streamGraph, jarFiles, globalClasspaths, userCodeClassLoader, savepointRestoreSettings) - .getJobExecutionResult(); + final PlanExecutor executor = PlanExecutor.createRemoteExecutor( + host, + port, + clientConfiguration, + jarFiles, + globalClasspaths); + return executor.executePlan(streamGraph).getJobExecutionResult(); } catch (ProgramInvocationException e) { throw e; @@ -318,7 +320,6 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment { @Deprecated protected JobExecutionResult executeRemotely(StreamGraph streamGraph, List jarFiles) throws ProgramInvocationException { return executeRemotely(streamGraph, - this.getClass().getClassLoader(), getConfig(), jarFiles, host, -- Gitee From 3cdf06b25b7ea9794768bf718d253bf222aa15b0 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 19 Sep 2019 16:42:43 +0200 Subject: [PATCH 238/268] [FLINK-14290] Use PipelineTranslationUtil and client.submitJob() in StreamContextEnvironment --- .../environment/StreamContextEnvironment.java | 20 ++++++++++++++++--- .../RemoteStreamExecutionEnvironmentTest.java | 10 +++++----- 2 files changed, 22 insertions(+), 8 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java index 6ee45411f6..b2db19b414 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java @@ -19,7 +19,10 @@ package org.apache.flink.streaming.api.environment; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.client.ClientUtils; +import org.apache.flink.client.FlinkPipelineTranslationUtil; import org.apache.flink.client.program.ContextEnvironment; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.streaming.api.graph.StreamGraph; /** @@ -43,9 +46,20 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment { public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { transformations.clear(); - // execute the programs - return ctx.getClient() - .run(streamGraph, ctx.getJars(), ctx.getClasspaths(), ctx.getUserCodeClassLoader(), ctx.getSavepointRestoreSettings()) + JobGraph jobGraph = FlinkPipelineTranslationUtil.getJobGraph( + streamGraph, + ctx.getClient().getFlinkConfiguration(), + getParallelism()); + + ClientUtils.addJarFiles(jobGraph, ctx.getJars()); + jobGraph.setClasspaths(ctx.getClasspaths()); + + // running from the CLI will override the savepoint restore settings + jobGraph.setSavepointRestoreSettings(ctx.getSavepointRestoreSettings()); + + return ctx + .getClient() + .submitJob(jobGraph, ctx.getUserCodeClassLoader()) .getJobExecutionResult(); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/RemoteStreamExecutionEnvironmentTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/RemoteStreamExecutionEnvironmentTest.java index b30c33844a..acfa6050be 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/RemoteStreamExecutionEnvironmentTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/RemoteStreamExecutionEnvironmentTest.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.api.environment; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.client.RemoteExecutor; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestOptions; @@ -39,7 +40,8 @@ import static org.mockito.Mockito.when; * Tests for the {@link RemoteStreamEnvironment}. */ @RunWith(PowerMockRunner.class) -@PrepareForTest({RemoteStreamEnvironment.class}) +// TODO: I don't like that I have to do this +@PrepareForTest({RemoteStreamEnvironment.class, RemoteExecutor.class}) public class RemoteStreamExecutionEnvironmentTest extends TestLogger { /** @@ -53,8 +55,7 @@ public class RemoteStreamExecutionEnvironmentTest extends TestLogger { JobExecutionResult expectedResult = new JobExecutionResult(null, 0, null); RestClusterClient mockedClient = Mockito.mock(RestClusterClient.class); - when(mockedClient.run(Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any())) - .thenReturn(expectedResult); + when(mockedClient.submitJob(Mockito.any(), Mockito.any())).thenReturn(expectedResult); PowerMockito.whenNew(RestClusterClient.class).withAnyArguments().thenAnswer((invocation) -> { Object[] args = invocation.getArguments(); @@ -85,8 +86,7 @@ public class RemoteStreamExecutionEnvironmentTest extends TestLogger { JobExecutionResult expectedResult = new JobExecutionResult(null, 0, null); PowerMockito.whenNew(RestClusterClient.class).withAnyArguments().thenReturn(mockedClient); - when(mockedClient.run(Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), Mockito.eq(restoreSettings))) - .thenReturn(expectedResult); + when(mockedClient.submitJob(Mockito.any(), Mockito.any())).thenReturn(expectedResult); JobExecutionResult actualResult = env.execute("fakeJobName"); Assert.assertEquals(expectedResult, actualResult); -- Gitee From ee8f9309d64e1438aef74e3a4376a0576f104412 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 24 Sep 2019 10:23:22 +0200 Subject: [PATCH 239/268] [hotfix] Fix formatting/checkstyle in PlanExecutor --- .../apache/flink/api/common/PlanExecutor.java | 117 ++++++++++-------- 1 file changed, 63 insertions(+), 54 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java b/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java index 83a8e17113..7c90383b21 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java @@ -29,15 +29,15 @@ import java.util.List; /** * A PlanExecutor executes a Flink program's dataflow plan. All Flink programs are translated to * dataflow plans prior to execution. - * + * *

    The specific implementation (such as the org.apache.flink.client.LocalExecutor - * and org.apache.flink.client.RemoteExecutor) determines where and how to run the dataflow. - * The concrete implementations of the executors are loaded dynamically, because they depend on - * the full set of all runtime classes.

    - * + * and org.apache.flink.client.RemoteExecutor) determines where and how to run the dataflow. The + * concrete implementations of the executors are loaded dynamically, because they depend on the full + * set of all runtime classes.

    + * *

    PlanExecutors can be started explicitly, in which case they keep running until stopped. If - * a program is submitted to a plan executor that is not running, it will start up for that - * program, and shut down afterwards.

    + * a program is submitted to a plan executor that is not running, it will start up for that program, + * and shut down afterwards.

    */ @Internal public abstract class PlanExecutor { @@ -48,19 +48,19 @@ public abstract class PlanExecutor { // ------------------------------------------------------------------------ // Program Execution // ------------------------------------------------------------------------ - + /** * Execute the given program. - * + * *

    If the executor has not been started before, then this method will start the - * executor and stop it after the execution has completed. This implies that one needs - * to explicitly start the executor for all programs where multiple dataflow parts - * depend on each other. Otherwise, the previous parts will no longer - * be available, because the executor immediately shut down after the execution.

    - * + * executor and stop it after the execution has completed. This implies that one needs to + * explicitly start the executor for all programs where multiple dataflow parts depend on each + * other. Otherwise, the previous parts will no longer be available, because the executor + * immediately shut down after the execution.

    + * * @param plan The plan of the program to execute. - * @return The execution result, containing for example the net runtime of the program, and the accumulators. - * + * @return The execution result, containing for example the net runtime of the program, and the + * accumulators. * @throws Exception Thrown, if job submission caused an exception. */ public abstract JobExecutionResult executePlan(Pipeline plan) throws Exception; @@ -68,77 +68,86 @@ public abstract class PlanExecutor { // ------------------------------------------------------------------------ // Executor Factories // ------------------------------------------------------------------------ - + /** * Creates an executor that runs the plan locally in a multi-threaded environment. - * + * * @return A local executor. */ public static PlanExecutor createLocalExecutor(Configuration configuration) { Class leClass = loadExecutorClass(LOCAL_EXECUTOR_CLASS); - + try { return leClass.getConstructor(Configuration.class).newInstance(configuration); - } - catch (Throwable t) { - throw new RuntimeException("An error occurred while loading the local executor (" - + LOCAL_EXECUTOR_CLASS + ").", t); + } catch (Throwable t) { + throw new RuntimeException( + "An error occurred while loading the local executor (" + LOCAL_EXECUTOR_CLASS + ").", + t); } } /** - * Creates an executor that runs the plan on a remote environment. The remote executor is typically used - * to send the program to a cluster for execution. + * Creates an executor that runs the plan on a remote environment. The remote executor is + * typically used to send the program to a cluster for execution. * * @param hostname The address of the JobManager to send the program to. * @param port The port of the JobManager to send the program to. * @param clientConfiguration The configuration for the client (Akka, default.parallelism). - * @param jarFiles A list of jar files that contain the user-defined function (UDF) classes and all classes used - * from within the UDFs. - * @param globalClasspaths A list of URLs that are added to the classpath of each user code classloader of the - * program. Paths must specify a protocol (e.g. file://) and be accessible on all nodes. + * @param jarFiles A list of jar files that contain the user-defined function (UDF) classes + * and all classes used from within the UDFs. + * @param globalClasspaths A list of URLs that are added to the classpath of each user code + * classloader of the program. Paths must specify a protocol (e.g. file://) and be + * accessible + * on all nodes. * @return A remote executor. */ - public static PlanExecutor createRemoteExecutor(String hostname, int port, Configuration clientConfiguration, - List jarFiles, List globalClasspaths) { + public static PlanExecutor createRemoteExecutor( + String hostname, + int port, + Configuration clientConfiguration, + List jarFiles, + List globalClasspaths) { if (hostname == null) { throw new IllegalArgumentException("The hostname must not be null."); } if (port <= 0 || port > 0xffff) { throw new IllegalArgumentException("The port value is out of range."); } - + Class reClass = loadExecutorClass(REMOTE_EXECUTOR_CLASS); - - List files = (jarFiles == null) ? - Collections.emptyList() : jarFiles; - List paths = (globalClasspaths == null) ? - Collections.emptyList() : globalClasspaths; + + List files = (jarFiles == null) ? Collections.emptyList() : jarFiles; + List paths = (globalClasspaths == null) ? Collections.emptyList() : + globalClasspaths; try { - return (clientConfiguration == null) ? - reClass.getConstructor(String.class, int.class, List.class) - .newInstance(hostname, port, files) : - reClass.getConstructor(String.class, int.class, Configuration.class, List.class, List.class) - .newInstance(hostname, port, clientConfiguration, files, paths); - } - catch (Throwable t) { - throw new RuntimeException("An error occurred while loading the remote executor (" - + REMOTE_EXECUTOR_CLASS + ").", t); + return (clientConfiguration == null) ? reClass + .getConstructor(String.class, int.class, List.class) + .newInstance(hostname, port, files) : reClass + .getConstructor(String.class, + int.class, + Configuration.class, + List.class, + List.class) + .newInstance(hostname, port, clientConfiguration, files, paths); + } catch (Throwable t) { + throw new RuntimeException( + "An error occurred while loading the remote executor (" + REMOTE_EXECUTOR_CLASS + ").", + t); } } - + private static Class loadExecutorClass(String className) { try { Class leClass = Class.forName(className); return leClass.asSubclass(PlanExecutor.class); - } - catch (ClassNotFoundException cnfe) { - throw new RuntimeException("Could not load the executor class (" + className - + "). Do you have the 'flink-clients' project in your dependencies?"); - } - catch (Throwable t) { - throw new RuntimeException("An error occurred while loading the executor (" + className + ").", t); + } catch (ClassNotFoundException cnfe) { + throw new RuntimeException("Could not load the executor class (" + className + "). Do " + + "you have the 'flink-clients' project in your dependencies?"); + } catch (Throwable t) { + throw new RuntimeException( + "An error occurred while loading the executor (" + className + ").", + t); } } } -- Gitee From eafdb64e40b30ca120e318a62be21107657b5573 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 24 Sep 2019 13:34:38 +0200 Subject: [PATCH 240/268] [FLINK-14290] Move jars and classpath out of executors and add to execute() method Before, the executors were potentially storing the jar and classpath internally while the jars and classpaths are really a property of the pipeline that should be executed. This change reflects that. --- .../apache/flink/client/LocalExecutor.java | 23 +++------- .../apache/flink/client/RemoteExecutor.java | 46 +++++++------------ .../RemoteExecutorHostnameResolutionTest.java | 8 ++-- .../apache/flink/api/common/PlanExecutor.java | 44 ++++++++---------- .../flink/api/java/LocalEnvironment.java | 7 ++- .../flink/api/java/RemoteEnvironment.java | 4 +- .../api/java/ScalaShellRemoteEnvironment.java | 4 +- .../apache/flink/api/java/FlinkILoopTest.java | 11 ++--- .../environment/LocalStreamEnvironment.java | 4 +- .../environment/RemoteStreamEnvironment.java | 7 ++- .../example/client/LocalExecutorITCase.java | 3 +- 11 files changed, 67 insertions(+), 94 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java index 290cff9d1c..d866a4f101 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java @@ -32,14 +32,15 @@ import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; import org.apache.flink.runtime.minicluster.RpcServiceSharing; -import java.util.Collections; +import java.net.URL; +import java.util.List; import static org.apache.flink.util.Preconditions.checkNotNull; /** * A PlanExecutor that runs Flink programs on a local embedded Flink runtime instance. * - *

    By simply calling the {@link #executePlan(Pipeline)} method, + *

    By simply calling the {@link #executePlan(Pipeline, List, List)} method, * this executor still start up and shut down again immediately after the program finished.

    * *

    To use this executor to execute many dataflow programs that constitute one job together, @@ -90,21 +91,11 @@ public class LocalExecutor extends PlanExecutor { return miniCluster; } - /** - * Executes the given program on a local runtime and waits for the job to finish. - * - *

    If the executor has not been started before, this starts the executor and shuts it down - * after the job finished. If the job runs in session mode, the executor is kept alive until - * no more references to the executor exist.

    - * - * @param pipeline The pipeline of the program to execute. - * @return The net runtime of the program, in milliseconds. - * - * @throws Exception Thrown, if either the startup of the local execution context, or the execution - * caused an exception. - */ @Override - public JobExecutionResult executePlan(Pipeline pipeline) throws Exception { + public JobExecutionResult executePlan( + Pipeline pipeline, + List jarFiles, + List globalClasspaths) throws Exception { checkNotNull(pipeline); // This is a quirk in how LocalEnvironment used to work. It sets the default parallelism diff --git a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java index d78a04fd3c..91a5d3a930 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java @@ -30,7 +30,6 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import java.net.InetSocketAddress; import java.net.URL; -import java.util.Collections; import java.util.List; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -47,35 +46,20 @@ import static org.apache.flink.util.Preconditions.checkNotNull; */ public class RemoteExecutor extends PlanExecutor { - private final List jarFiles; - - private final List globalClasspaths; - private final Configuration clientConfiguration; private int defaultParallelism = 1; public RemoteExecutor(String hostname, int port) { - this(hostname, port, new Configuration(), Collections.emptyList(), Collections.emptyList()); + this(hostname, port, new Configuration()); } - public RemoteExecutor( - String hostname, - int port, - Configuration clientConfiguration, - List jarFiles, - List globalClasspaths) { - this(new InetSocketAddress(hostname, port), clientConfiguration, jarFiles, globalClasspaths); + public RemoteExecutor(String hostname, int port, Configuration clientConfiguration) { + this(new InetSocketAddress(hostname, port), clientConfiguration); } - public RemoteExecutor( - InetSocketAddress inet, - Configuration clientConfiguration, - List jarFiles, - List globalClasspaths) { + public RemoteExecutor(InetSocketAddress inet, Configuration clientConfiguration) { this.clientConfiguration = clientConfiguration; - this.jarFiles = jarFiles; - this.globalClasspaths = globalClasspaths; clientConfiguration.setString(JobManagerOptions.ADDRESS, inet.getHostName()); clientConfiguration.setInteger(JobManagerOptions.PORT, inet.getPort()); @@ -87,8 +71,8 @@ public class RemoteExecutor extends PlanExecutor { // ------------------------------------------------------------------------ /** - * Sets the parallelism that will be used when neither the program does not define - * any parallelism at all. + * Sets the parallelism that will be used when neither the program does not define any + * parallelism at all. * * @param defaultParallelism The default parallelism for the executor. */ @@ -100,8 +84,8 @@ public class RemoteExecutor extends PlanExecutor { } /** - * Gets the parallelism that will be used when neither the program does not define - * any parallelism at all. + * Gets the parallelism that will be used when neither the program does not define any + * parallelism at all. * * @return The default parallelism for the executor. */ @@ -114,11 +98,13 @@ public class RemoteExecutor extends PlanExecutor { // ------------------------------------------------------------------------ @Override - public JobExecutionResult executePlan(Pipeline plan) throws Exception { + public JobExecutionResult executePlan( + Pipeline plan, + List jarFiles, + List globalClasspaths) throws Exception { checkNotNull(plan); - JobGraph jobGraph = FlinkPipelineTranslationUtil.getJobGraph( - plan, + JobGraph jobGraph = FlinkPipelineTranslationUtil.getJobGraph(plan, clientConfiguration, getDefaultParallelism()); @@ -127,7 +113,7 @@ public class RemoteExecutor extends PlanExecutor { ClassLoader userCodeClassLoader = ClientUtils.buildUserCodeClassLoader( jarFiles, - this.globalClasspaths, + globalClasspaths, getClass().getClassLoader()); return executePlanWithJars(jobGraph, userCodeClassLoader); @@ -137,7 +123,9 @@ public class RemoteExecutor extends PlanExecutor { checkNotNull(jobGraph); checkNotNull(classLoader); - try (ClusterClient client = new RestClusterClient<>(clientConfiguration, "RemoteExecutor")) { + try (ClusterClient client = new RestClusterClient<>( + clientConfiguration, + "RemoteExecutor")) { return client.submitJob(jobGraph, classLoader).getJobExecutionResult(); } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java b/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java index 73e99e58b7..187c345a3f 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java @@ -29,7 +29,6 @@ import org.junit.Test; import java.net.InetAddress; import java.net.InetSocketAddress; -import java.net.URL; import java.net.UnknownHostException; import java.util.Collections; @@ -54,7 +53,7 @@ public class RemoteExecutorHostnameResolutionTest extends TestLogger { RemoteExecutor exec = new RemoteExecutor(nonExistingHostname, port); try { - exec.executePlan(getProgram()); + exec.executePlan(getProgram(), Collections.emptyList(), Collections.emptyList()); fail("This should fail with an ProgramInvocationException"); } catch (UnknownHostException ignored) { @@ -66,10 +65,9 @@ public class RemoteExecutorHostnameResolutionTest extends TestLogger { public void testUnresolvableHostname2() throws Exception { InetSocketAddress add = new InetSocketAddress(nonExistingHostname, port); - RemoteExecutor exec = new RemoteExecutor(add, new Configuration(), - Collections.emptyList(), Collections.emptyList()); + RemoteExecutor exec = new RemoteExecutor(add, new Configuration()); try { - exec.executePlan(getProgram()); + exec.executePlan(getProgram(), Collections.emptyList(), Collections.emptyList()); fail("This should fail with an ProgramInvocationException"); } catch (UnknownHostException ignored) { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java b/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java index 7c90383b21..2af3874c54 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/PlanExecutor.java @@ -23,7 +23,6 @@ import org.apache.flink.api.dag.Pipeline; import org.apache.flink.configuration.Configuration; import java.net.URL; -import java.util.Collections; import java.util.List; /** @@ -59,11 +58,20 @@ public abstract class PlanExecutor { * immediately shut down after the execution.

    * * @param plan The plan of the program to execute. + * @param jarFiles A list of jar files that contain the user-defined function (UDF) classes + * and all classes used from within the UDFs. + * @param globalClasspaths A list of URLs that are added to the classpath of each user code + * classloader of the program. Paths must specify a protocol (e.g. file://) and be + * accessible + * on all nodes. * @return The execution result, containing for example the net runtime of the program, and the * accumulators. * @throws Exception Thrown, if job submission caused an exception. */ - public abstract JobExecutionResult executePlan(Pipeline plan) throws Exception; + public abstract JobExecutionResult executePlan( + Pipeline plan, + List jarFiles, + List globalClasspaths) throws Exception; // ------------------------------------------------------------------------ // Executor Factories @@ -93,20 +101,10 @@ public abstract class PlanExecutor { * @param hostname The address of the JobManager to send the program to. * @param port The port of the JobManager to send the program to. * @param clientConfiguration The configuration for the client (Akka, default.parallelism). - * @param jarFiles A list of jar files that contain the user-defined function (UDF) classes - * and all classes used from within the UDFs. - * @param globalClasspaths A list of URLs that are added to the classpath of each user code - * classloader of the program. Paths must specify a protocol (e.g. file://) and be - * accessible - * on all nodes. * @return A remote executor. */ public static PlanExecutor createRemoteExecutor( - String hostname, - int port, - Configuration clientConfiguration, - List jarFiles, - List globalClasspaths) { + String hostname, int port, Configuration clientConfiguration) { if (hostname == null) { throw new IllegalArgumentException("The hostname must not be null."); } @@ -116,20 +114,14 @@ public abstract class PlanExecutor { Class reClass = loadExecutorClass(REMOTE_EXECUTOR_CLASS); - List files = (jarFiles == null) ? Collections.emptyList() : jarFiles; - List paths = (globalClasspaths == null) ? Collections.emptyList() : - globalClasspaths; - try { - return (clientConfiguration == null) ? reClass - .getConstructor(String.class, int.class, List.class) - .newInstance(hostname, port, files) : reClass - .getConstructor(String.class, - int.class, - Configuration.class, - List.class, - List.class) - .newInstance(hostname, port, clientConfiguration, files, paths); + if (clientConfiguration == null) { + return reClass.getConstructor(String.class, int.class).newInstance(hostname, port); + } else { + return reClass + .getConstructor(String.class, int.class, Configuration.class) + .newInstance(hostname, port, clientConfiguration); + } } catch (Throwable t) { throw new RuntimeException( "An error occurred while loading the remote executor (" + REMOTE_EXECUTOR_CLASS + ").", diff --git a/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java index 9ce09467b1..dc886507a6 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java @@ -26,6 +26,8 @@ import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.PlanExecutor; import org.apache.flink.configuration.Configuration; +import java.util.Collections; + import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -73,7 +75,10 @@ public class LocalEnvironment extends ExecutionEnvironment { final Plan p = createProgramPlan(jobName); final PlanExecutor executor = PlanExecutor.createLocalExecutor(configuration); - lastJobExecutionResult = executor.executePlan(p); + lastJobExecutionResult = executor.executePlan( + p, + Collections.emptyList(), + Collections.emptyList()); return lastJobExecutionResult; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java index 786add1400..9b6add7666 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java @@ -155,8 +155,8 @@ public class RemoteEnvironment extends ExecutionEnvironment { public JobExecutionResult execute(String jobName) throws Exception { final Plan p = createProgramPlan(jobName); - final PlanExecutor executor = PlanExecutor.createRemoteExecutor(host, port, clientConfiguration, jarFiles, globalClasspaths); - lastJobExecutionResult = executor.executePlan(p); + final PlanExecutor executor = PlanExecutor.createRemoteExecutor(host, port, clientConfiguration); + lastJobExecutionResult = executor.executePlan(p, jarFiles, globalClasspaths); return lastJobExecutionResult; } diff --git a/flink-scala-shell/src/main/java/org/apache/flink/api/java/ScalaShellRemoteEnvironment.java b/flink-scala-shell/src/main/java/org/apache/flink/api/java/ScalaShellRemoteEnvironment.java index 6898a99a2e..e545192742 100644 --- a/flink-scala-shell/src/main/java/org/apache/flink/api/java/ScalaShellRemoteEnvironment.java +++ b/flink-scala-shell/src/main/java/org/apache/flink/api/java/ScalaShellRemoteEnvironment.java @@ -62,8 +62,8 @@ public class ScalaShellRemoteEnvironment extends RemoteEnvironment { final Plan p = createProgramPlan(jobName); final List allJarFiles = getUpdatedJarFiles(); - final PlanExecutor executor = PlanExecutor.createRemoteExecutor(host, port, clientConfiguration, allJarFiles, globalClasspaths); - lastJobExecutionResult = executor.executePlan(p); + final PlanExecutor executor = PlanExecutor.createRemoteExecutor(host, port, clientConfiguration); + lastJobExecutionResult = executor.executePlan(p, allJarFiles, globalClasspaths); return lastJobExecutionResult; } diff --git a/flink-scala-shell/src/test/java/org/apache/flink/api/java/FlinkILoopTest.java b/flink-scala-shell/src/test/java/org/apache/flink/api/java/FlinkILoopTest.java index 5b46c54093..19bc2a0aa2 100644 --- a/flink-scala-shell/src/test/java/org/apache/flink/api/java/FlinkILoopTest.java +++ b/flink-scala-shell/src/test/java/org/apache/flink/api/java/FlinkILoopTest.java @@ -39,6 +39,7 @@ import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import java.net.URL; import java.util.List; import scala.Option; @@ -68,18 +69,13 @@ public class FlinkILoopTest extends TestLogger { BDDMockito.given(PlanExecutor.createRemoteExecutor( Matchers.anyString(), Matchers.anyInt(), - Matchers.any(Configuration.class), - Matchers.any(java.util.List.class), - Matchers.any(java.util.List.class) + Matchers.any(Configuration.class) )).willAnswer(new Answer() { @Override public PlanExecutor answer(InvocationOnMock invocation) throws Throwable { testPlanExecutor.setHost((String) invocation.getArguments()[0]); testPlanExecutor.setPort((Integer) invocation.getArguments()[1]); testPlanExecutor.setConfiguration((Configuration) invocation.getArguments()[2]); - testPlanExecutor.setJars((List) invocation.getArguments()[3]); - testPlanExecutor.setGlobalClasspaths((List) invocation.getArguments()[4]); - return testPlanExecutor; } }); @@ -128,7 +124,8 @@ public class FlinkILoopTest extends TestLogger { private List globalClasspaths; @Override - public JobExecutionResult executePlan(Pipeline plan) throws Exception { + public JobExecutionResult executePlan( + Pipeline plan, List jarFiles, List globalClasspaths) throws Exception { return null; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java index 54f4354591..a896cf2ed5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java @@ -27,6 +27,8 @@ import org.apache.flink.streaming.api.graph.StreamGraph; import javax.annotation.Nonnull; +import java.util.Collections; + /** * The LocalStreamEnvironment is a StreamExecutionEnvironment that runs the program locally, * multi-threaded, in the JVM where the environment is instantiated. It spawns an embedded @@ -76,7 +78,7 @@ public class LocalStreamEnvironment extends StreamExecutionEnvironment { public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { try { final PlanExecutor executor = PlanExecutor.createLocalExecutor(configuration); - return executor.executePlan(streamGraph); + return executor.executePlan(streamGraph, Collections.emptyList(), Collections.emptyList()); } finally { transformations.clear(); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java index b393e5ee3d..1de834aaa7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java @@ -279,10 +279,9 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment { final PlanExecutor executor = PlanExecutor.createRemoteExecutor( host, port, - clientConfiguration, - jarFiles, - globalClasspaths); - return executor.executePlan(streamGraph).getJobExecutionResult(); + clientConfiguration); + + return executor.executePlan(streamGraph, jarFiles, globalClasspaths).getJobExecutionResult(); } catch (ProgramInvocationException e) { throw e; diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java index af76ec9cec..331f6a98cb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java @@ -34,6 +34,7 @@ import org.junit.Test; import java.io.File; import java.io.FileWriter; +import java.util.Collections; /** * Integration tests for {@link LocalExecutor}. @@ -62,7 +63,7 @@ public class LocalExecutorITCase extends TestLogger { Plan wcPlan = getWordCountPlan(inFile, outFile, parallelism); wcPlan.setExecutionConfig(new ExecutionConfig()); - executor.executePlan(wcPlan); + executor.executePlan(wcPlan, Collections.emptyList(), Collections.emptyList()); } catch (Exception e) { e.printStackTrace(); Assert.fail(e.getMessage()); -- Gitee From 98b54e0d9d2bd3ade8ca10d2a70fb5cb7a20a4f7 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 10 Oct 2019 17:18:20 +0200 Subject: [PATCH 241/268] [FLINK-14391] Factor out translator discovery in FlinkPipelineTranslationUtil This way, we can reuse the discovery from different methods that we'll add in follow-up commits. --- .../client/FlinkPipelineTranslationUtil.java | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java index 33c8027708..e54a4280d5 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java +++ b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java @@ -38,13 +38,18 @@ public final class FlinkPipelineTranslationUtil { Configuration optimizerConfiguration, int defaultParallelism) { + FlinkPipelineTranslator pipelineTranslator = getPipelineTranslator(pipeline); + + return pipelineTranslator.translate(pipeline, + optimizerConfiguration, + defaultParallelism); + } + + private static FlinkPipelineTranslator getPipelineTranslator(Pipeline pipeline) { PlanTranslator planToJobGraphTransmogrifier = new PlanTranslator(); if (planToJobGraphTransmogrifier.canTranslate(pipeline)) { - // we have a DataSet program - return planToJobGraphTransmogrifier.translate(pipeline, - optimizerConfiguration, - defaultParallelism); + return planToJobGraphTransmogrifier; } FlinkPipelineTranslator streamGraphTranslator = reflectStreamGraphTranslator(); @@ -53,10 +58,7 @@ public final class FlinkPipelineTranslationUtil { throw new RuntimeException("Translator " + streamGraphTranslator + " cannot translate " + "the given pipeline " + pipeline + "."); } - - return streamGraphTranslator.translate(pipeline, - optimizerConfiguration, - defaultParallelism); + return streamGraphTranslator; } private static FlinkPipelineTranslator reflectStreamGraphTranslator() { -- Gitee From 70a8e0c407a21174f608adf8ff1d4254d9024490 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 11 Oct 2019 14:08:10 +0200 Subject: [PATCH 242/268] [FLINK-14391] Add JobID setter in JobGraph This allows setting the JobID after creation of the JobGraph. --- .../java/org/apache/flink/runtime/jobgraph/JobGraph.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java index 377f870ca0..3c6f935753 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java @@ -68,7 +68,7 @@ public class JobGraph implements Serializable { private final Configuration jobConfiguration = new Configuration(); /** ID of this job. May be set if specific job id is desired (e.g. session management) */ - private final JobID jobID; + private JobID jobID; /** Name of this job. */ private final String jobName; @@ -189,6 +189,13 @@ public class JobGraph implements Serializable { return this.jobID; } + /** + * Sets the ID of the job. + */ + public void setJobID(JobID jobID) { + this.jobID = jobID; + } + /** * Returns the name assigned to the job graph. * -- Gitee From 7084e07eaba0d967cb2474bb4ea9e80da5cca46e Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 11 Oct 2019 14:17:43 +0200 Subject: [PATCH 243/268] [FLINK-14391] Add JSON execution graph generation to PipelineTranslationUtil --- .../client/FlinkPipelineTranslationUtil.java | 10 +++++++++- .../flink/client/FlinkPipelineTranslator.java | 8 +++++++- .../apache/flink/client/PlanTranslator.java | 19 ++++++++++++++++++- .../api/graph/StreamGraphTranslator.java | 14 ++++++++++++-- 4 files changed, 46 insertions(+), 5 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java index e54a4280d5..f88c69dd52 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java +++ b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java @@ -40,11 +40,19 @@ public final class FlinkPipelineTranslationUtil { FlinkPipelineTranslator pipelineTranslator = getPipelineTranslator(pipeline); - return pipelineTranslator.translate(pipeline, + return pipelineTranslator.translateToJobGraph(pipeline, optimizerConfiguration, defaultParallelism); } + /** + * Extracts the execution plan (as JSON) from the given {@link Pipeline}. + */ + public static String translateToJSONExecutionPlan(Pipeline pipeline) { + FlinkPipelineTranslator pipelineTranslator = getPipelineTranslator(pipeline); + return pipelineTranslator.translateToJSONExecutionPlan(pipeline); + } + private static FlinkPipelineTranslator getPipelineTranslator(Pipeline pipeline) { PlanTranslator planToJobGraphTransmogrifier = new PlanTranslator(); diff --git a/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslator.java b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslator.java index c224422599..fa3bc398f4 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslator.java +++ b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslator.java @@ -33,10 +33,16 @@ public interface FlinkPipelineTranslator { * Creates a {@link JobGraph} from the given {@link Pipeline} and attaches the given jar * files and classpaths to the {@link JobGraph}. */ - JobGraph translate( + JobGraph translateToJobGraph( Pipeline pipeline, Configuration optimizerConfiguration, int defaultParallelism); + + /** + * Extracts the execution plan (as JSON) from the given {@link Pipeline}. + */ + String translateToJSONExecutionPlan(Pipeline pipeline); + boolean canTranslate(Pipeline pipeline); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/PlanTranslator.java b/flink-clients/src/main/java/org/apache/flink/client/PlanTranslator.java index ae18b6b24a..43c6bb7299 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/PlanTranslator.java +++ b/flink-clients/src/main/java/org/apache/flink/client/PlanTranslator.java @@ -24,7 +24,9 @@ import org.apache.flink.api.dag.Pipeline; import org.apache.flink.configuration.Configuration; import org.apache.flink.optimizer.DataStatistics; import org.apache.flink.optimizer.Optimizer; +import org.apache.flink.optimizer.costs.DefaultCostEstimator; import org.apache.flink.optimizer.plan.OptimizedPlan; +import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -41,7 +43,7 @@ public class PlanTranslator implements FlinkPipelineTranslator { private static final Logger LOG = LoggerFactory.getLogger(PlanTranslator.class); @Override - public JobGraph translate( + public JobGraph translateToJobGraph( Pipeline pipeline, Configuration optimizerConfiguration, int defaultParallelism) { @@ -67,6 +69,21 @@ public class PlanTranslator implements FlinkPipelineTranslator { plan.getDefaultParallelism()); } + @Override + public String translateToJSONExecutionPlan(Pipeline pipeline) { + checkArgument(pipeline instanceof Plan, "Given pipeline is not a DataSet Plan."); + + Plan plan = (Plan) pipeline; + + Optimizer opt = new Optimizer( + new DataStatistics(), + new DefaultCostEstimator(), + new Configuration()); + OptimizedPlan optPlan = opt.compile(plan); + + return new PlanJSONDumpGenerator().getOptimizerPlanAsJSON(optPlan); + } + private JobGraph compilePlan(Plan plan, Configuration optimizerConfiguration) { Optimizer optimizer = new Optimizer(new DataStatistics(), optimizerConfiguration); OptimizedPlan optimizedPlan = optimizer.compile(plan); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphTranslator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphTranslator.java index ebd554fc9c..4a43adbb2e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphTranslator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphTranslator.java @@ -41,7 +41,7 @@ public class StreamGraphTranslator implements FlinkPipelineTranslator { private static final Logger LOG = LoggerFactory.getLogger(StreamGraphTranslator.class); @Override - public JobGraph translate( + public JobGraph translateToJobGraph( Pipeline pipeline, Configuration optimizerConfiguration, int defaultParallelism) { @@ -49,7 +49,17 @@ public class StreamGraphTranslator implements FlinkPipelineTranslator { "Given pipeline is not a DataStream StreamGraph."); StreamGraph streamGraph = (StreamGraph) pipeline; - return streamGraph.getJobGraph(); + return streamGraph.getJobGraph(null); + } + + @Override + public String translateToJSONExecutionPlan(Pipeline pipeline) { + checkArgument(pipeline instanceof StreamGraph, + "Given pipeline is not a DataStream StreamGraph."); + + StreamGraph streamGraph = (StreamGraph) pipeline; + + return streamGraph.getStreamingPlanAsJSON(); } @Override -- Gitee From 835e776a9038448333a05b6e08e7f036ae2c74be Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 11 Oct 2019 14:25:58 +0200 Subject: [PATCH 244/268] [FLINK-14391] Remove JobID parameter from exception in RemoteStreamEnvironment Creating a JobGraph from a StreamGraph using this method creates a random JobID that doesn't give any information. --- .../streaming/api/environment/RemoteStreamEnvironment.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java index 1de834aaa7..2574b8e343 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java @@ -267,8 +267,7 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment { client = new RestClusterClient<>(configuration, "RemoteStreamEnvironment"); } catch (Exception e) { - throw new ProgramInvocationException("Cannot establish connection to JobManager: " + e.getMessage(), - streamGraph.getJobGraph().getJobID(), e); + throw new ProgramInvocationException("Cannot establish connection to JobManager: " + e.getMessage(), e); } if (savepointRestoreSettings != null) { @@ -288,8 +287,7 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment { } catch (Exception e) { String term = e.getMessage() == null ? "." : (": " + e.getMessage()); - throw new ProgramInvocationException("The program execution failed" + term, - streamGraph.getJobGraph().getJobID(), e); + throw new ProgramInvocationException("The program execution failed" + term, e); } finally { try { -- Gitee From 59dd855628052c369b64c71edc1018ed378e8eec Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 11 Oct 2019 14:20:39 +0200 Subject: [PATCH 245/268] [FLINK-14391] Remove FlinkPlan as common base class of OptimizerPlan and StreamGraph We also need to change/simplify some translation logic because of this. --- .../apache/flink/client/cli/CliFrontend.java | 20 +--- .../flink/client/program/ClusterClient.java | 109 ------------------ .../program/OptimizerPlanEnvironment.java | 28 ++--- .../client/program/PackagedProgramUtils.java | 60 +++++----- .../cli/CliFrontendPackageProgramTest.java | 7 +- .../flink/client/program/ClientTest.java | 24 ++-- .../program/ExecutionPlanCreationTest.java | 4 +- .../flink/optimizer/plan/FlinkPlan.java | 28 ----- .../flink/optimizer/plan/OptimizedPlan.java | 2 +- .../flink/optimizer/plan/StreamingPlan.java | 45 -------- .../environment/StreamPlanEnvironment.java | 2 +- .../streaming/api/graph/StreamGraph.java | 14 ++- .../gateway/local/ExecutionContext.java | 40 ++++--- 13 files changed, 102 insertions(+), 281 deletions(-) delete mode 100644 flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/FlinkPlan.java delete mode 100644 flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/StreamingPlan.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java index c4d5fa9b3e..552eccc7f9 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java @@ -24,6 +24,8 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.api.common.accumulators.AccumulatorHelper; +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.client.FlinkPipelineTranslationUtil; import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; @@ -40,13 +42,6 @@ import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.plugin.PluginUtils; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.costs.DefaultCostEstimator; -import org.apache.flink.optimizer.plan.FlinkPlan; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.StreamingPlan; -import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -335,15 +330,8 @@ public class CliFrontend { LOG.info("Creating program plan dump"); - Optimizer compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), configuration); - FlinkPlan flinkPlan = ClusterClient.getOptimizedPlan(compiler, program, parallelism); - - String jsonPlan = null; - if (flinkPlan instanceof OptimizedPlan) { - jsonPlan = new PlanJSONDumpGenerator().getOptimizerPlanAsJSON((OptimizedPlan) flinkPlan); - } else if (flinkPlan instanceof StreamingPlan) { - jsonPlan = ((StreamingPlan) flinkPlan).getStreamingPlanAsJSON(); - } + Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram(program, parallelism); + String jsonPlan = FlinkPipelineTranslationUtil.translateToJSONExecutionPlan(pipeline); if (jsonPlan != null) { System.out.println("----------------------- Execution Plan -----------------------"); diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index 8179403212..c0ea516c93 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -21,22 +21,10 @@ package org.apache.flink.client.program; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; -import org.apache.flink.api.common.Plan; import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.Path; -import org.apache.flink.optimizer.CompilerException; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.costs.DefaultCostEstimator; -import org.apache.flink.optimizer.plan.FlinkPlan; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.StreamingPlan; -import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.util.FlinkException; @@ -49,7 +37,6 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; import javax.annotation.Nullable; -import java.net.URISyntaxException; import java.net.URL; import java.util.Collection; import java.util.List; @@ -65,9 +52,6 @@ public abstract class ClusterClient implements AutoCloseable { protected final Logger log = LoggerFactory.getLogger(getClass()); - /** The optimizer used in the optimization of batch programs. */ - final Optimizer compiler; - /** Configuration of the client. */ private final Configuration flinkConfig; @@ -94,7 +78,6 @@ public abstract class ClusterClient implements AutoCloseable { */ public ClusterClient(Configuration flinkConfig) { this.flinkConfig = Preconditions.checkNotNull(flinkConfig); - this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), flinkConfig); } /** @@ -110,45 +93,6 @@ public abstract class ClusterClient implements AutoCloseable { } - // ------------------------------------------------------------------------ - // Access to the Program's Plan - // ------------------------------------------------------------------------ - - public static String getOptimizedPlanAsJson(Optimizer compiler, PackagedProgram prog, int parallelism) - throws CompilerException, ProgramInvocationException { - PlanJSONDumpGenerator jsonGen = new PlanJSONDumpGenerator(); - return jsonGen.getOptimizerPlanAsJSON((OptimizedPlan) getOptimizedPlan(compiler, prog, parallelism)); - } - - public static FlinkPlan getOptimizedPlan(Optimizer compiler, PackagedProgram prog, int parallelism) - throws CompilerException, ProgramInvocationException { - final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); - try { - Thread.currentThread().setContextClassLoader(prog.getUserCodeClassLoader()); - - // temporary hack to support the optimizer plan preview - OptimizerPlanEnvironment env = new OptimizerPlanEnvironment(compiler); - if (parallelism > 0) { - env.setParallelism(parallelism); - } - return env.getOptimizedPlan(prog); - } finally { - Thread.currentThread().setContextClassLoader(contextClassLoader); - } - } - - public static OptimizedPlan getOptimizedPlan(Optimizer compiler, Plan p, int parallelism) throws CompilerException { - Logger log = LoggerFactory.getLogger(ClusterClient.class); - - if (parallelism > 0 && p.getDefaultParallelism() <= 0) { - log.debug("Changing plan default parallelism from {} to {}", p.getDefaultParallelism(), parallelism); - p.setDefaultParallelism(parallelism); - } - log.debug("Set parallelism {}, plan default parallelism {}", parallelism, p.getDefaultParallelism()); - - return compiler.compile(p); - } - // ------------------------------------------------------------------------ // Program submission / execution // ------------------------------------------------------------------------ @@ -193,28 +137,6 @@ public abstract class ClusterClient implements AutoCloseable { } } - public JobSubmissionResult run( - Plan plan, - List libraries, - List classpaths, - ClassLoader classLoader, - int parallelism, - SavepointRestoreSettings savepointSettings) throws CompilerException, ProgramInvocationException { - - OptimizedPlan optPlan = getOptimizedPlan(compiler, plan, parallelism); - return run(optPlan, libraries, classpaths, classLoader, savepointSettings); - } - - public JobSubmissionResult run( - FlinkPlan compiledPlan, - List libraries, - List classpaths, - ClassLoader classLoader, - SavepointRestoreSettings savepointSettings) throws ProgramInvocationException { - JobGraph job = getJobGraph(flinkConfig, compiledPlan, libraries, classpaths, savepointSettings); - return submitJob(job, classLoader); - } - /** * Requests the {@link JobStatus} of the job with the given {@link JobID}. */ @@ -293,37 +215,6 @@ public abstract class ClusterClient implements AutoCloseable { */ public abstract Map> getAccumulators(JobID jobID, ClassLoader loader) throws Exception; - // ------------------------------------------------------------------------ - // Internal translation methods - // ------------------------------------------------------------------------ - - public static JobGraph getJobGraph(Configuration flinkConfig, PackagedProgram prog, FlinkPlan optPlan, SavepointRestoreSettings savepointSettings) throws ProgramInvocationException { - return getJobGraph(flinkConfig, optPlan, prog.getAllLibraries(), prog.getClasspaths(), savepointSettings); - } - - public static JobGraph getJobGraph(Configuration flinkConfig, FlinkPlan optPlan, List jarFiles, List classpaths, SavepointRestoreSettings savepointSettings) { - JobGraph job; - if (optPlan instanceof StreamingPlan) { - job = ((StreamingPlan) optPlan).getJobGraph(); - job.setSavepointRestoreSettings(savepointSettings); - } else { - JobGraphGenerator gen = new JobGraphGenerator(flinkConfig); - job = gen.compileJobGraph((OptimizedPlan) optPlan); - } - - for (URL jar : jarFiles) { - try { - job.addJar(new Path(jar.toURI())); - } catch (URISyntaxException e) { - throw new RuntimeException("URL is invalid. This should not happen.", e); - } - } - - job.setClasspaths(classpaths); - - return job; - } - // ------------------------------------------------------------------------ // Abstract methods to be implemented by the cluster specific Client // ------------------------------------------------------------------------ diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java index d2801ac060..edf7d36595 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java @@ -19,27 +19,20 @@ package org.apache.flink.client.program; import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.Plan; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironmentFactory; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.plan.FlinkPlan; import java.io.ByteArrayOutputStream; import java.io.PrintStream; /** - * An {@link ExecutionEnvironment} that never executes a job but only creates the optimized plan. + * An {@link ExecutionEnvironment} that never executes a job but only extracts the {@link + * org.apache.flink.api.dag.Pipeline}. */ public class OptimizerPlanEnvironment extends ExecutionEnvironment { - private final Optimizer compiler; - - private FlinkPlan optimizerPlan; - - public OptimizerPlanEnvironment(Optimizer compiler) { - this.compiler = compiler; - } + private Pipeline pipeline; // ------------------------------------------------------------------------ // Execution Environment methods @@ -47,14 +40,13 @@ public class OptimizerPlanEnvironment extends ExecutionEnvironment { @Override public JobExecutionResult execute(String jobName) throws Exception { - Plan plan = createProgramPlan(jobName); - this.optimizerPlan = compiler.compile(plan); + this.pipeline = createProgramPlan(); // do not go on with anything now! throw new ProgramAbortException(); } - public FlinkPlan getOptimizedPlan(PackagedProgram prog) throws ProgramInvocationException { + public Pipeline getPipeline(PackagedProgram prog) throws ProgramInvocationException { // temporarily write syserr and sysout to a byte array. PrintStream originalOut = System.out; @@ -73,8 +65,8 @@ public class OptimizerPlanEnvironment extends ExecutionEnvironment { } catch (Throwable t) { // the invocation gets aborted with the preview plan - if (optimizerPlan != null) { - return optimizerPlan; + if (pipeline != null) { + return pipeline; } else { throw new ProgramInvocationException("The program caused an error: ", t); } @@ -112,8 +104,8 @@ public class OptimizerPlanEnvironment extends ExecutionEnvironment { // ------------------------------------------------------------------------ - public void setPlan(FlinkPlan plan){ - this.optimizerPlan = plan; + public void setPipeline(Pipeline pipeline){ + this.pipeline = pipeline; } /** diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java index d9a10e647b..fa9f8b0486 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java @@ -19,22 +19,15 @@ package org.apache.flink.client.program; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.client.ClientUtils; +import org.apache.flink.client.FlinkPipelineTranslationUtil; import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.Path; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.costs.DefaultCostEstimator; -import org.apache.flink.optimizer.plan.FlinkPlan; -import org.apache.flink.optimizer.plan.OptimizedPlan; -import org.apache.flink.optimizer.plan.StreamingPlan; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; +import org.apache.flink.optimizer.CompilerException; import org.apache.flink.runtime.jobgraph.JobGraph; import javax.annotation.Nullable; -import java.net.URISyntaxException; -import java.net.URL; - /** * Utility class for {@link PackagedProgram} related operations. */ @@ -56,33 +49,21 @@ public class PackagedProgramUtils { Configuration configuration, int defaultParallelism, @Nullable JobID jobID) throws ProgramInvocationException { + Thread.currentThread().setContextClassLoader(packagedProgram.getUserCodeClassLoader()); - final Optimizer optimizer = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), configuration); - final OptimizerPlanEnvironment optimizerPlanEnvironment = new OptimizerPlanEnvironment(optimizer); + final OptimizerPlanEnvironment optimizerPlanEnvironment = new OptimizerPlanEnvironment(); optimizerPlanEnvironment.setParallelism(defaultParallelism); + final Pipeline pipeline = optimizerPlanEnvironment.getPipeline(packagedProgram); - final FlinkPlan flinkPlan = optimizerPlanEnvironment.getOptimizedPlan(packagedProgram); - - final JobGraph jobGraph; + final JobGraph jobGraph = FlinkPipelineTranslationUtil.getJobGraph(pipeline, configuration, defaultParallelism); - if (flinkPlan instanceof StreamingPlan) { - jobGraph = ((StreamingPlan) flinkPlan).getJobGraph(jobID); - jobGraph.setSavepointRestoreSettings(packagedProgram.getSavepointSettings()); - } else { - final JobGraphGenerator jobGraphGenerator = new JobGraphGenerator(configuration); - jobGraph = jobGraphGenerator.compileJobGraph((OptimizedPlan) flinkPlan, jobID); - } - - for (URL url : packagedProgram.getAllLibraries()) { - try { - jobGraph.addJar(new Path(url.toURI())); - } catch (URISyntaxException e) { - throw new ProgramInvocationException("Invalid URL for jar file: " + url + '.', jobGraph.getJobID(), e); - } + if (jobID != null) { + jobGraph.setJobID(jobID); } - + ClientUtils.addJarFiles(jobGraph, packagedProgram.getAllLibraries()); jobGraph.setClasspaths(packagedProgram.getClasspaths()); + jobGraph.setSavepointRestoreSettings(packagedProgram.getSavepointSettings()); return jobGraph; } @@ -104,5 +85,22 @@ public class PackagedProgramUtils { return createJobGraph(packagedProgram, configuration, defaultParallelism, null); } + public static Pipeline getPipelineFromProgram(PackagedProgram prog, int parallelism) + throws CompilerException, ProgramInvocationException { + final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(prog.getUserCodeClassLoader()); + + // temporary hack to support the optimizer plan preview + OptimizerPlanEnvironment env = new OptimizerPlanEnvironment(); + if (parallelism > 0) { + env.setParallelism(parallelism); + } + return env.getPipeline(prog); + } finally { + Thread.currentThread().setContextClassLoader(contextClassLoader); + } + } + private PackagedProgramUtils() {} } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java index 48c8891203..873ba00417 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java @@ -18,8 +18,10 @@ package org.apache.flink.client.cli; -import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.client.FlinkPipelineTranslationUtil; import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.program.PackagedProgramUtils; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.Configuration; import org.apache.flink.optimizer.DataStatistics; @@ -284,7 +286,8 @@ public class CliFrontendPackageProgramTest extends TestLogger { Optimizer compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), c); // we expect this to fail with a "ClassNotFoundException" - ClusterClient.getOptimizedPlanAsJson(compiler, prog, 666); + Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram(prog, 666); + FlinkPipelineTranslationUtil.translateToJSONExecutionPlan(pipeline); fail("Should have failed with a ClassNotFoundException"); } catch (ProgramInvocationException e) { diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java index b31f95cc93..634ebf0c28 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java @@ -28,6 +28,8 @@ 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.ClientUtils; +import org.apache.flink.client.FlinkPipelineTranslationUtil; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; @@ -36,7 +38,7 @@ import org.apache.flink.optimizer.Optimizer; import org.apache.flink.optimizer.costs.DefaultCostEstimator; import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.testutils.MiniClusterResource; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; @@ -156,13 +158,16 @@ public class ClientTest extends TestLogger { public void shouldSubmitToJobClient() throws Exception { final ClusterClient clusterClient = new MiniClusterClient(new Configuration(), MINI_CLUSTER_RESOURCE.getMiniCluster()); clusterClient.setDetached(true); - JobSubmissionResult result = clusterClient.run( - plan, - Collections.emptyList(), - Collections.emptyList(), - getClass().getClassLoader(), - 1, - SavepointRestoreSettings.none()); + + JobGraph jobGraph = FlinkPipelineTranslationUtil.getJobGraph( + plan, + new Configuration(), + 1); + + ClientUtils.addJarFiles(jobGraph, Collections.emptyList()); + jobGraph.setClasspaths(Collections.emptyList()); + + JobSubmissionResult result = clusterClient.submitJob(jobGraph, getClass().getClassLoader()); assertNotNull(result); } @@ -198,7 +203,8 @@ public class ClientTest extends TestLogger { PackagedProgram prg = new PackagedProgram(TestOptimizerPlan.class, "/dev/random", "/tmp"); Optimizer optimizer = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config); - OptimizedPlan op = (OptimizedPlan) ClusterClient.getOptimizedPlan(optimizer, prg, 1); + Plan plan = (Plan) PackagedProgramUtils.getPipelineFromProgram(prg, 1); + OptimizedPlan op = optimizer.compile(plan); assertNotNull(op); PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator(); diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java index d26b0d0010..1b52f377c7 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java @@ -18,6 +18,7 @@ package org.apache.flink.client.program; +import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.ProgramDescription; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.DataSet; @@ -59,7 +60,8 @@ public class ExecutionPlanCreationTest { config.setInteger(JobManagerOptions.PORT, mockJmAddress.getPort()); Optimizer optimizer = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config); - OptimizedPlan op = (OptimizedPlan) ClusterClient.getOptimizedPlan(optimizer, prg, -1); + Plan plan = (Plan) PackagedProgramUtils.getPipelineFromProgram(prg, -1); + OptimizedPlan op = optimizer.compile(plan); assertNotNull(op); PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator(); diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/FlinkPlan.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/FlinkPlan.java deleted file mode 100644 index d146c83e35..0000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/FlinkPlan.java +++ /dev/null @@ -1,28 +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.optimizer.plan; - -/** - * A common interface for compiled Flink plans for both batch and streaming - * processing programs. - * - */ -public interface FlinkPlan { - -} diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/OptimizedPlan.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/OptimizedPlan.java index 311c2861e7..3e8f2f02f6 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/OptimizedPlan.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/OptimizedPlan.java @@ -32,7 +32,7 @@ import org.apache.flink.util.Visitor; * all operator strategies (sorting-merge join, hash join, sorted grouping, ...), * and the data exchange modes (batched, pipelined).

    */ -public class OptimizedPlan implements FlinkPlan, Visitable { +public class OptimizedPlan implements Visitable { /** The data sources in the plan. */ private final Collection dataSources; diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/StreamingPlan.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/StreamingPlan.java deleted file mode 100644 index f3fe632207..0000000000 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/StreamingPlan.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.optimizer.plan; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.jobgraph.JobGraph; - -import javax.annotation.Nullable; - -/** - * Abstract class representing Flink Streaming plans. - */ -public abstract class StreamingPlan implements FlinkPlan { - - /** - * Gets the assembled {@link JobGraph} with a random {@link JobID}. - */ - @SuppressWarnings("deprecation") - public JobGraph getJobGraph() { - return getJobGraph(null); - } - - /** - * Gets the assembled {@link JobGraph} with a specified {@link JobID}. - */ - public abstract JobGraph getJobGraph(@Nullable JobID jobID); - - public abstract String getStreamingPlanAsJSON(); -} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java index bb1a4cfe0b..54ef3e2923 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java @@ -57,7 +57,7 @@ public class StreamPlanEnvironment extends StreamExecutionEnvironment { transformations.clear(); if (env instanceof OptimizerPlanEnvironment) { - ((OptimizerPlanEnvironment) env).setPlan(streamGraph); + ((OptimizerPlanEnvironment) env).setPipeline(streamGraph); } throw new OptimizerPlanEnvironment.ProgramAbortException(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java index ec6992fd7a..a8b322f81e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java @@ -31,7 +31,6 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.MissingTypeInfo; -import org.apache.flink.optimizer.plan.StreamingPlan; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.jobgraph.ScheduleMode; @@ -73,7 +72,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; * */ @Internal -public class StreamGraph extends StreamingPlan implements Pipeline { +public class StreamGraph implements Pipeline { private static final Logger LOG = LoggerFactory.getLogger(StreamGraph.class); @@ -727,14 +726,19 @@ public class StreamGraph extends StreamingPlan implements Pipeline { } /** - * Gets the assembled {@link JobGraph} with a given job id. + * Gets the assembled {@link JobGraph} with a random {@link JobID}. + */ + public JobGraph getJobGraph() { + return getJobGraph(null); + } + + /** + * Gets the assembled {@link JobGraph} with a specified {@link JobID}. */ - @Override public JobGraph getJobGraph(@Nullable JobID jobID) { return StreamingJobGraphGenerator.createJobGraph(this, jobID); } - @Override public String getStreamingPlanAsJSON() { try { return new JSONGenerator(this).getJSON(); diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java index 27c3ee261a..73c9ccebd9 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -21,19 +21,17 @@ package org.apache.flink.table.client.gateway.local; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.client.ClientUtils; +import org.apache.flink.client.FlinkPipelineTranslationUtil; import org.apache.flink.client.cli.CliArgsException; import org.apache.flink.client.cli.CustomCommandLine; import org.apache.flink.client.cli.RunOptions; import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.deployment.ClusterSpecification; -import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.plugin.TemporaryClassLoaderContext; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.costs.DefaultCostEstimator; -import org.apache.flink.optimizer.plan.FlinkPlan; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.streaming.api.TimeCharacteristic; @@ -440,16 +438,29 @@ public class ExecutionContext { } public JobGraph createJobGraph(String name) { - final FlinkPlan plan = createPlan(name, flinkConfig); - return ClusterClient.getJobGraph( - flinkConfig, - plan, - dependencies, - runOptions.getClasspaths(), - runOptions.getSavepointRestoreSettings()); + final Pipeline pipeline = createPipeline(name, flinkConfig); + + int parallelism; + if (execEnv != null) { + parallelism = execEnv.getParallelism(); + } else if (streamExecEnv != null) { + parallelism = streamExecEnv.getParallelism(); + } else { + throw new RuntimeException("No execution environment defined."); + } + JobGraph jobGraph = FlinkPipelineTranslationUtil.getJobGraph( + pipeline, + flinkConfig, + parallelism); + + ClientUtils.addJarFiles(jobGraph, dependencies); + jobGraph.setClasspaths(runOptions.getClasspaths()); + jobGraph.setSavepointRestoreSettings(runOptions.getSavepointRestoreSettings()); + + return jobGraph; } - private FlinkPlan createPlan(String name, Configuration flinkConfig) { + private Pipeline createPipeline(String name, Configuration flinkConfig) { if (streamExecEnv != null) { // special case for Blink planner to apply batch optimizations // note: it also modifies the ExecutionConfig! @@ -461,8 +472,7 @@ public class ExecutionContext { final int parallelism = execEnv.getParallelism(); final Plan unoptimizedPlan = execEnv.createProgramPlan(); unoptimizedPlan.setJobName(name); - final Optimizer compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), flinkConfig); - return ClusterClient.getOptimizedPlan(compiler, unoptimizedPlan, parallelism); + return unoptimizedPlan; } } -- Gitee From 41492c409aebd821069f19c721ae3cb64a91758e Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 8 Oct 2019 11:26:55 +0200 Subject: [PATCH 246/268] [hotfix][hs] Deduplicate variables --- .../webmonitor/history/HistoryServerArchiveFetcher.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java index fed220fe95..4cd8b89059 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java @@ -152,7 +152,6 @@ class HistoryServerArchiveFetcher { if (jobArchives == null) { continue; } - boolean updateOverview = false; int numFetchedArchives = 0; for (FileStatus jobArchive : jobArchives) { Path jobArchivePath = jobArchive.getPath(); @@ -200,7 +199,6 @@ class HistoryServerArchiveFetcher { fw.flush(); } } - updateOverview = true; numFetchedArchives++; } catch (IOException e) { LOG.error("Failure while fetching/processing job archive for job {}.", jobID, e); @@ -223,7 +221,7 @@ class HistoryServerArchiveFetcher { } } } - if (updateOverview) { + if (numFetchedArchives > 0) { updateJobOverview(webOverviewDir, webDir); for (int x = 0; x < numFetchedArchives; x++) { numArchivedJobs.countDown(); -- Gitee From f22f1eba8f7695857a2015ed178365191849dac4 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 8 Oct 2019 11:32:59 +0200 Subject: [PATCH 247/268] [hotfix][hs] Clarify write access to webJobDir --- .../runtime/webmonitor/history/HistoryServerArchiveFetcher.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java index 4cd8b89059..6f52cef1b1 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java @@ -176,6 +176,7 @@ class HistoryServerArchiveFetcher { json = convertLegacyJobOverview(json); target = new File(webOverviewDir, jobID + JSON_FILE_ENDING); } else { + // this implicitly writes into webJobDir target = new File(webDir, path + JSON_FILE_ENDING); } -- Gitee From 26bc3c8c65c757285c58b2cfcb0ba81111395ea4 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 8 Oct 2019 10:48:39 +0200 Subject: [PATCH 248/268] [FLINK-14337][hs] Prevent NPE on corrupt archives --- .../flink/runtime/history/FsJobArchivist.java | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/history/FsJobArchivist.java b/flink-runtime/src/main/java/org/apache/flink/runtime/history/FsJobArchivist.java index d0fbc5eb8c..ab1e34d740 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/history/FsJobArchivist.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/history/FsJobArchivist.java @@ -109,15 +109,20 @@ public class FsJobArchivist { ByteArrayOutputStream output = new ByteArrayOutputStream()) { IOUtils.copyBytes(input, output); - JsonNode archive = mapper.readTree(output.toByteArray()); + try { + JsonNode archive = mapper.readTree(output.toByteArray()); - Collection archives = new ArrayList<>(); - for (JsonNode archivePart : archive.get(ARCHIVE)) { - String path = archivePart.get(PATH).asText(); - String json = archivePart.get(JSON).asText(); - archives.add(new ArchivedJson(path, json)); + Collection archives = new ArrayList<>(); + for (JsonNode archivePart : archive.get(ARCHIVE)) { + String path = archivePart.get(PATH).asText(); + String json = archivePart.get(JSON).asText(); + archives.add(new ArchivedJson(path, json)); + } + return archives; + } catch (NullPointerException npe) { + // occurs if the archive is empty or any of the expected fields are not present + throw new IOException("Job archive (" + file.getPath() + ") did not conform to expected format."); } - return archives; } } } -- Gitee From 88ae9f8e3a5749e262fc2a9217ced4f2dc997b44 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 8 Oct 2019 11:27:34 +0200 Subject: [PATCH 249/268] [FLINK-14337][hs] Only mark archives as processed on success --- .../webmonitor/history/HistoryServerArchiveFetcher.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java index 6f52cef1b1..79add41fba 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java @@ -163,7 +163,7 @@ class HistoryServerArchiveFetcher { refreshDir, jobID, iae); continue; } - if (cachedArchives.add(jobID)) { + if (!cachedArchives.contains(jobID)) { try { for (ArchivedJson archive : FsJobArchivist.getArchivedJsons(jobArchive.getPath())) { String path = archive.getPath(); @@ -200,11 +200,10 @@ class HistoryServerArchiveFetcher { fw.flush(); } } + cachedArchives.add(jobID); numFetchedArchives++; } catch (IOException e) { LOG.error("Failure while fetching/processing job archive for job {}.", jobID, e); - // Make sure we attempt to fetch the archive again - cachedArchives.remove(jobID); // Make sure we do not include this job in the overview try { Files.delete(new File(webOverviewDir, jobID + JSON_FILE_ENDING).toPath()); -- Gitee From 8c7b2d8cf47a14e201f3b96701436d74bb98a2c8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cyrille=20Ch=C3=A9p=C3=A9lov?= Date: Thu, 4 Jul 2019 10:07:13 +0200 Subject: [PATCH 250/268] [FLINK-13097] Make the cause for EOFException explicit (buffer depletion) --- .../flink/runtime/io/disk/SimpleCollectingOutputView.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java index 1a45ff2a83..2fa6c883c6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java @@ -76,7 +76,7 @@ public class SimpleCollectingOutputView extends AbstractPagedOutputView { this.segmentNum++; return next; } else { - throw new EOFException(); + throw new EOFException("Can't collect further: memorySource depleted"); } } -- Gitee From 36e9b2d8e4b7490e0de5cf35e15471981c4c43b8 Mon Sep 17 00:00:00 2001 From: wangpeibin Date: Thu, 27 Jun 2019 11:31:01 +0800 Subject: [PATCH 251/268] [FLINK-13008] fix findbugs warning in AggregationsFunction --- .../flink/table/runtime/aggregate/GeneratedAggregations.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/aggregate/GeneratedAggregations.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/aggregate/GeneratedAggregations.scala index b771c5e3b9..f2b506fa6d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/aggregate/GeneratedAggregations.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/aggregate/GeneratedAggregations.scala @@ -149,7 +149,7 @@ class SingleElementIterable[T] extends java.lang.Iterable[T] { } } - override def remove(): Unit = new java.lang.UnsupportedOperationException + override def remove(): Unit = throw new java.lang.UnsupportedOperationException } val it = new SingleElementIterator -- Gitee From ff249bce3dd4ef61518fb1e647e31e8f85640b61 Mon Sep 17 00:00:00 2001 From: cyq89051127 Date: Sun, 25 Aug 2019 15:01:43 +0800 Subject: [PATCH 252/268] [FLINK-12979][formats] Allow empty line delimiter for CsvRowSerializationSchema This closes #9529. --- docs/dev/table/connect.md | 7 ++++--- docs/dev/table/connect.zh.md | 11 ++++++----- .../flink/formats/csv/CsvRowSerializationSchema.java | 4 ++-- .../java/org/apache/flink/table/descriptors/Csv.java | 2 +- .../apache/flink/table/descriptors/CsvValidator.java | 2 +- .../formats/csv/CsvRowDeSerializationSchemaTest.java | 11 +++++++++++ 6 files changed, 25 insertions(+), 12 deletions(-) diff --git a/docs/dev/table/connect.md b/docs/dev/table/connect.md index 975a0d01d8..520662026e 100644 --- a/docs/dev/table/connect.md +++ b/docs/dev/table/connect.md @@ -1355,7 +1355,7 @@ The CSV format can be used as follows: .fieldDelimiter(';') // optional: field delimiter character (',' by default) .lineDelimiter("\r\n") // optional: line delimiter ("\n" by default; - // otherwise "\r" or "\r\n" are allowed) + // otherwise "\r", "\r\n", or "" are allowed) .quoteCharacter('\'') // optional: quote character for enclosing field values ('"' by default) .allowComments() // optional: ignores comment lines that start with '#' (disabled by default); // if enabled, make sure to also ignore parse errors to allow empty rows @@ -1383,7 +1383,7 @@ The CSV format can be used as follows: .field_delimiter(';') # optional: field delimiter character (',' by default) .line_delimiter("\r\n") # optional: line delimiter ("\n" by default; - # otherwise "\r" or "\r\n" are allowed) + # otherwise "\r", "\r\n", or "" are allowed) .quote_character('\'') # optional: quote character for enclosing field values ('"' by default) .allow_comments() # optional: ignores comment lines that start with '#' (disabled by default); # if enabled, make sure to also ignore parse errors to allow empty rows @@ -1410,7 +1410,8 @@ format: derive-schema: true field-delimiter: ";" # optional: field delimiter character (',' by default) - line-delimiter: "\r\n" # optional: line delimiter ("\n" by default; otherwise "\r" or "\r\n" are allowed) + line-delimiter: "\r\n" # optional: line delimiter ("\n" by default; + # otherwise "\r", "\r\n", or "" are allowed) quote-character: "'" # optional: quote character for enclosing field values ('"' by default) allow-comments: true # optional: ignores comment lines that start with "#" (disabled by default); # if enabled, make sure to also ignore parse errors to allow empty rows diff --git a/docs/dev/table/connect.zh.md b/docs/dev/table/connect.zh.md index 6deb6af554..9040cf1cba 100644 --- a/docs/dev/table/connect.zh.md +++ b/docs/dev/table/connect.zh.md @@ -1114,7 +1114,7 @@ The CSV format can be used as follows: .fieldDelimiter(';') // optional: field delimiter character (',' by default) .lineDelimiter("\r\n") // optional: line delimiter ("\n" by default; - // otherwise "\r" or "\r\n" are allowed) + // otherwise "\r", "\r\n", or "" are allowed) .quoteCharacter('\'') // optional: quote character for enclosing field values ('"' by default) .allowComments() // optional: ignores comment lines that start with '#' (disabled by default); // if enabled, make sure to also ignore parse errors to allow empty rows @@ -1142,9 +1142,9 @@ The CSV format can be used as follows: .field_delimiter(';') # optional: field delimiter character (',' by default) .line_delimiter("\r\n") # optional: line delimiter ("\n" by default; - # otherwise "\r" or "\r\n" are allowed) - .quote_character('\'') # optional: quote character for enclosing field values ('"' by default) - .allow_comments() # optional: ignores comment lines that start with '#' (disabled by default); + # otherwise "\r", "\r\n", or "" are allowed) + .quote_character("'") # optional: quote character for enclosing field values ('"' by default) + .allow_comments() # optional: ignores comment lines that start with "#" (disabled by default); # if enabled, make sure to also ignore parse errors to allow empty rows .ignore_parse_errors() # optional: skip fields and rows with parse errors instead of failing; # fields are set to null in case of errors @@ -1169,7 +1169,8 @@ format: derive-schema: true field-delimiter: ";" # optional: field delimiter character (',' by default) - line-delimiter: "\r\n" # optional: line delimiter ("\n" by default; otherwise "\r" or "\r\n" are allowed) + line-delimiter: "\r\n" # optional: line delimiter ("\n" by default; + # otherwise "\r", "\r\n", or "" are allowed) quote-character: "'" # optional: quote character for enclosing field values ('"' by default) allow-comments: true # optional: ignores comment lines that start with "#" (disabled by default); # if enabled, make sure to also ignore parse errors to allow empty rows diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowSerializationSchema.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowSerializationSchema.java index a65554f8ca..b3c2de4934 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowSerializationSchema.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowSerializationSchema.java @@ -116,9 +116,9 @@ public final class CsvRowSerializationSchema implements SerializationSchema public Builder setLineDelimiter(String delimiter) { Preconditions.checkNotNull(delimiter, "Delimiter must not be null."); - if (!delimiter.equals("\n") && !delimiter.equals("\r") && !delimiter.equals("\r\n")) { + if (!delimiter.equals("\n") && !delimiter.equals("\r") && !delimiter.equals("\r\n") && !delimiter.equals("")) { throw new IllegalArgumentException( - "Unsupported new line delimiter. Only \\n, \\r, or \\r\\n are supported."); + "Unsupported new line delimiter. Only \\n, \\r, \\r\\n, or empty string are supported."); } this.csvSchema = this.csvSchema.rebuild().setLineSeparator(delimiter).build(); return this; diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/Csv.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/Csv.java index c467f1f002..5fb32050bb 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/Csv.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/Csv.java @@ -75,7 +75,7 @@ public class Csv extends FormatDescriptor { } /** - * Sets the line delimiter ("\n" by default; otherwise "\r" or "\r\n" are allowed). + * Sets the line delimiter ("\n" by default; otherwise "\r", "\r\n", or "" are allowed). * * @param delimiter the line delimiter */ diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/CsvValidator.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/CsvValidator.java index bc53971681..2f7ebabc19 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/CsvValidator.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/CsvValidator.java @@ -44,7 +44,7 @@ public class CsvValidator extends FormatDescriptorValidator { public void validate(DescriptorProperties properties) { super.validate(properties); properties.validateString(FORMAT_FIELD_DELIMITER, true, 1, 1); - properties.validateEnumValues(FORMAT_LINE_DELIMITER, true, Arrays.asList("\r", "\n", "\r\n")); + properties.validateEnumValues(FORMAT_LINE_DELIMITER, true, Arrays.asList("\r", "\n", "\r\n", "")); properties.validateString(FORMAT_QUOTE_CHARACTER, true, 1, 1); properties.validateBoolean(FORMAT_ALLOW_COMMENTS, true); properties.validateBoolean(FORMAT_IGNORE_PARSE_ERRORS, true); diff --git a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvRowDeSerializationSchemaTest.java b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvRowDeSerializationSchemaTest.java index ece2e45e18..02b3e2782f 100644 --- a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvRowDeSerializationSchemaTest.java +++ b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvRowDeSerializationSchemaTest.java @@ -168,6 +168,17 @@ public class CsvRowDeSerializationSchemaTest { serialize(serSchemaBuilder, Row.of("Test", 12, "Hello"))); } + @Test + public void testEmptyLineDelimiter() throws Exception { + final TypeInformation rowInfo = Types.ROW(Types.STRING, Types.INT, Types.STRING); + final CsvRowSerializationSchema.Builder serSchemaBuilder = new CsvRowSerializationSchema.Builder(rowInfo) + .setLineDelimiter(""); + + assertArrayEquals( + "Test,12,Hello".getBytes(), + serialize(serSchemaBuilder, Row.of("Test", 12, "Hello"))); + } + @Test(expected = IllegalArgumentException.class) public void testInvalidNesting() throws Exception { testNullableField(Types.ROW(Types.ROW(Types.STRING)), "FAIL", Row.of(Row.of("FAIL"))); -- Gitee From 8e81fc265d95f634401269b2acf9a4e80c0c1044 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Wed, 16 Oct 2019 18:24:01 +0800 Subject: [PATCH 253/268] [FLINK-14409][table] Fix MapType and MultisetType doesn't accept any subclass of java.util.Map for inputs This closes #9913. --- .../flink/table/types/logical/MapType.java | 3 +++ .../table/types/logical/MultisetType.java | 3 +++ .../table/expressions/ExpressionTest.java | 19 +++++++++++++++++++ .../flink/table/types/LogicalTypesTest.java | 8 +++++--- 4 files changed, 30 insertions(+), 3 deletions(-) diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MapType.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MapType.java index 4d7726584e..cd0ab71814 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MapType.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MapType.java @@ -91,6 +91,9 @@ public final class MapType extends LogicalType { @Override public boolean supportsInputConversion(Class clazz) { + if (Map.class.isAssignableFrom(clazz)) { + return true; + } return INPUT_OUTPUT_CONVERSION.contains(clazz.getName()); } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MultisetType.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MultisetType.java index 54aa7e4010..cd4b7ee6c2 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MultisetType.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MultisetType.java @@ -82,6 +82,9 @@ public final class MultisetType extends LogicalType { @Override public boolean supportsInputConversion(Class clazz) { + if (Map.class.isAssignableFrom(clazz)) { + return true; + } return INPUT_OUTPUT_CONVERSION.contains(clazz.getName()); } diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/expressions/ExpressionTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/expressions/ExpressionTest.java index eb80d25bc0..c9d265b58f 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/expressions/ExpressionTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/expressions/ExpressionTest.java @@ -30,6 +30,8 @@ import org.junit.rules.ExpectedException; import java.math.BigDecimal; import java.sql.Timestamp; import java.time.LocalDateTime; +import java.util.HashMap; +import java.util.Map; import static java.util.Arrays.asList; import static java.util.Collections.singletonList; @@ -86,6 +88,23 @@ public class ExpressionTest { new String[][]{null, null, {"1", "2", "3", "Dog's"}}, DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.STRING()))) .toString()); + + final Map map = new HashMap<>(); + map.put("key1", 1); + map.put("key2", 2); + map.put("key3", 3); + assertEquals( + "{key1=1, key2=2, key3=3}", + new ValueLiteralExpression( + map, + DataTypes.MAP(DataTypes.STRING(), DataTypes.INT())) + .toString()); + assertEquals( + "{key1=1, key2=2, key3=3}", + new ValueLiteralExpression( + map, + DataTypes.MULTISET(DataTypes.STRING())) + .toString()); } @Test diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypesTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypesTest.java index 6bbbc80c11..5ce9937f6d 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypesTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypesTest.java @@ -68,7 +68,9 @@ import org.junit.Test; import java.math.BigDecimal; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.Map; +import java.util.TreeMap; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -426,7 +428,7 @@ public class LogicalTypesTest { new MultisetType(new TimestampType()), "MULTISET", "MULTISET", - new Class[]{Map.class}, + new Class[]{Map.class, HashMap.class, TreeMap.class}, new Class[]{Map.class}, new LogicalType[]{new TimestampType()}, new MultisetType(new SmallIntType()) @@ -436,7 +438,7 @@ public class LogicalTypesTest { new MultisetType(new MultisetType(new TimestampType())), "MULTISET>", "MULTISET>", - new Class[]{Map.class}, + new Class[]{Map.class, HashMap.class, TreeMap.class}, new Class[]{Map.class}, new LogicalType[]{new MultisetType(new TimestampType())}, new MultisetType(new MultisetType(new SmallIntType())) @@ -449,7 +451,7 @@ public class LogicalTypesTest { new MapType(new VarCharType(20), new TimestampType()), "MAP", "MAP", - new Class[]{Map.class}, + new Class[]{Map.class, HashMap.class, TreeMap.class}, new Class[]{Map.class}, new LogicalType[]{new VarCharType(20), new TimestampType()}, new MapType(new VarCharType(99), new TimestampType()) -- Gitee From 94b444e3ec128e7a4daaaf4a49ceeeca9d886f1d Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 16 Oct 2019 17:33:39 +0200 Subject: [PATCH 254/268] [hotfix][docs] Add MAP data type docs --- docs/dev/table/types.md | 38 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/docs/dev/table/types.md b/docs/dev/table/types.md index 13139ff9a1..8a913bcfd6 100644 --- a/docs/dev/table/types.md +++ b/docs/dev/table/types.md @@ -1005,6 +1005,43 @@ equivalent to `ARRAY`. |:----------|:-----:|:------:|:----------------------------------| |*t*`[]` | (X) | (X) | Depends on the subtype. *Default* | +#### `MAP` + +Data type of an associative array that maps keys (including `NULL`) to values (including `NULL`). A map +cannot contain duplicate keys; each key can map to at most one value. + +There is no restriction of element types; it is the responsibility of the user to ensure uniqueness. + +The map type is an extension to the SQL standard. + +**Declaration** + +
    + +
    +{% highlight text %} +MAP +{% endhighlight %} +
    + +
    +{% highlight java %} +DataTypes.MAP(kt, vt) +{% endhighlight %} +
    + +
    + +The type can be declared using `MAP` where `kt` is the data type of the key elements +and `vt` is the data type of the value elements. + +**Bridging to JVM Types** + +| Java Type | Input | Output | Remarks | +|:--------------------------------------|:-----:|:------:|:----------| +| `java.util.Map` | X | X | *Default* | +| *subclass* of `java.util.Map` | X | | | + #### `MULTISET` Data type of a multiset (=bag). Unlike a set, it allows for multiple instances for each of its @@ -1042,6 +1079,7 @@ equivalent to `MULTISET`. | Java Type | Input | Output | Remarks | |:-------------------------------------|:-----:|:------:|:---------------------------------------------------------| |`java.util.Map` | X | X | Assigns each value to an integer multiplicity. *Default* | +| *subclass* of `java.util.Map` | X | | Assigns each value to an integer multiplicity. | #### `ROW` -- Gitee From ad052a870d1a1d457721a67ec4bbea4bc49d33c5 Mon Sep 17 00:00:00 2001 From: wangxlong <18868816710@163.com> Date: Thu, 17 Oct 2019 10:47:08 +0800 Subject: [PATCH 255/268] [FLINK-14421][table] Add 'L' suffix to static long value (#9917) --- .../flink/table/expressions/utils/ApiExpressionUtils.java | 2 +- .../java/org/apache/calcite/avatica/util/DateTimeUtils.java | 2 +- .../apache/flink/table/runtime/functions/SqlDateTimeUtils.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/utils/ApiExpressionUtils.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/utils/ApiExpressionUtils.java index 5d0c2c365e..99de217042 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/utils/ApiExpressionUtils.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/utils/ApiExpressionUtils.java @@ -50,7 +50,7 @@ public final class ApiExpressionUtils { public static final long MILLIS_PER_HOUR = 3600000L; // = 60 * 60 * 1000 - public static final long MILLIS_PER_DAY = 86400000; // = 24 * 60 * 60 * 1000 + public static final long MILLIS_PER_DAY = 86400000L; // = 24 * 60 * 60 * 1000 private ApiExpressionUtils() { // private diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/avatica/util/DateTimeUtils.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/avatica/util/DateTimeUtils.java index fec3b4fa1b..315ffaf633 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/avatica/util/DateTimeUtils.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/avatica/util/DateTimeUtils.java @@ -89,7 +89,7 @@ public class DateTimeUtils { *

    This is the modulo 'mask' used when converting * TIMESTAMP values to DATE and TIME values. */ - public static final long MILLIS_PER_DAY = 86400000; // = 24 * 60 * 60 * 1000 + public static final long MILLIS_PER_DAY = 86400000L; // = 24 * 60 * 60 * 1000 /** * Calendar set to the epoch (1970-01-01 00:00:00 UTC). Useful for diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java index a6784bbfd6..e7f1422a94 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java @@ -77,7 +77,7 @@ public class SqlDateTimeUtils { *

    This is the modulo 'mask' used when converting * TIMESTAMP values to DATE and TIME values. */ - private static final long MILLIS_PER_DAY = 86400000; // = 24 * 60 * 60 * 1000 + private static final long MILLIS_PER_DAY = 86400000L; // = 24 * 60 * 60 * 1000 /** The SimpleDateFormat string for ISO dates, "yyyy-MM-dd". */ private static final String DATE_FORMAT_STRING = "yyyy-MM-dd"; -- Gitee From 456a17685374aaa2f17479c013fde56fc528e7fb Mon Sep 17 00:00:00 2001 From: Jiayi Liao Date: Sun, 6 Oct 2019 14:30:57 +0800 Subject: [PATCH 256/268] [FLINK-14296][sql-parser] Improve handling of parameters nullabillity in parser module 1. Add @Nullable annotation to nullable fields 2. Use Optional instead of nullable as return value 3. Add requrieNonNull check in constructor for non-null fields 4. List fields do not need to check against null This closes #9843 --- .../src/main/codegen/includes/parserImpls.ftl | 4 +- .../flink/sql/parser/ddl/SqlCreateTable.java | 108 +++++++++--------- .../flink/sql/parser/ddl/SqlCreateView.java | 17 ++- .../flink/sql/parser/ddl/SqlTableColumn.java | 9 +- .../sql/parser/FlinkSqlParserImplTest.java | 12 ++ .../operations/SqlToOperationConverter.java | 37 +++--- .../sqlexec/SqlToOperationConverter.java | 37 +++--- 7 files changed, 111 insertions(+), 113 deletions(-) diff --git a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl index 06f137b474..6c86e06dba 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl +++ b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl @@ -173,8 +173,8 @@ SqlCreate SqlCreateTable(Span s, boolean replace) : { final SqlParserPos startPos = s.pos(); SqlIdentifier tableName; - SqlNodeList primaryKeyList = null; - List uniqueKeysList = null; + SqlNodeList primaryKeyList = SqlNodeList.EMPTY; + List uniqueKeysList = new ArrayList(); SqlNodeList columnList = SqlNodeList.EMPTY; SqlCharStringLiteral comment = null; diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java index 4fdd43370f..ba0763938d 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java @@ -37,9 +37,12 @@ import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.pretty.SqlPrettyWriter; import org.apache.calcite.util.ImmutableNullableList; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; import static java.util.Objects.requireNonNull; @@ -63,6 +66,7 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { private final SqlNodeList partitionKeyList; + @Nullable private final SqlCharStringLiteral comment; public SqlCreateTable( @@ -75,12 +79,12 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { SqlNodeList partitionKeyList, SqlCharStringLiteral comment) { super(OPERATOR, pos, false, false); - this.tableName = requireNonNull(tableName, "Table name is missing"); - this.columnList = requireNonNull(columnList, "Column list should not be null"); - this.primaryKeyList = primaryKeyList; - this.uniqueKeysList = uniqueKeysList; - this.propertyList = propertyList; - this.partitionKeyList = partitionKeyList; + this.tableName = requireNonNull(tableName, "tableName should not be null"); + this.columnList = requireNonNull(columnList, "columnList should not be null"); + this.primaryKeyList = requireNonNull(primaryKeyList, "primayKeyList should not be null"); + this.uniqueKeysList = requireNonNull(uniqueKeysList, "uniqueKeysList should not be null"); + this.propertyList = requireNonNull(propertyList, "propertyList should not be null"); + this.partitionKeyList = requireNonNull(partitionKeyList, "partitionKeyList should not be null"); this.comment = comment; } @@ -119,8 +123,8 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { return uniqueKeysList; } - public SqlCharStringLiteral getComment() { - return comment; + public Optional getComment() { + return Optional.ofNullable(comment); } public boolean isIfNotExists() { @@ -129,60 +133,52 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { public void validate() throws SqlValidateException { Set columnNames = new HashSet<>(); - if (columnList != null) { - for (SqlNode column : columnList) { - String columnName = null; - if (column instanceof SqlTableColumn) { - SqlTableColumn tableColumn = (SqlTableColumn) column; - columnName = tableColumn.getName().getSimple(); - } else if (column instanceof SqlBasicCall) { - SqlBasicCall tableColumn = (SqlBasicCall) column; - columnName = tableColumn.getOperands()[1].toString(); - } + for (SqlNode column : columnList) { + String columnName = null; + if (column instanceof SqlTableColumn) { + SqlTableColumn tableColumn = (SqlTableColumn) column; + columnName = tableColumn.getName().getSimple(); + } else if (column instanceof SqlBasicCall) { + SqlBasicCall tableColumn = (SqlBasicCall) column; + columnName = tableColumn.getOperands()[1].toString(); + } - if (!columnNames.add(columnName)) { - throw new SqlValidateException( - column.getParserPosition(), - "Duplicate column name [" + columnName + "], at " + - column.getParserPosition()); - } + if (!columnNames.add(columnName)) { + throw new SqlValidateException( + column.getParserPosition(), + "Duplicate column name [" + columnName + "], at " + + column.getParserPosition()); } } - if (this.primaryKeyList != null) { - for (SqlNode primaryKeyNode : this.primaryKeyList) { - String primaryKey = ((SqlIdentifier) primaryKeyNode).getSimple(); - if (!columnNames.contains(primaryKey)) { - throw new SqlValidateException( - primaryKeyNode.getParserPosition(), - "Primary key [" + primaryKey + "] not defined in columns, at " + - primaryKeyNode.getParserPosition()); - } + for (SqlNode primaryKeyNode : this.primaryKeyList) { + String primaryKey = ((SqlIdentifier) primaryKeyNode).getSimple(); + if (!columnNames.contains(primaryKey)) { + throw new SqlValidateException( + primaryKeyNode.getParserPosition(), + "Primary key [" + primaryKey + "] not defined in columns, at " + + primaryKeyNode.getParserPosition()); } } - if (this.uniqueKeysList != null) { - for (SqlNodeList uniqueKeys: this.uniqueKeysList) { - for (SqlNode uniqueKeyNode : uniqueKeys) { - String uniqueKey = ((SqlIdentifier) uniqueKeyNode).getSimple(); - if (!columnNames.contains(uniqueKey)) { - throw new SqlValidateException( - uniqueKeyNode.getParserPosition(), - "Unique key [" + uniqueKey + "] not defined in columns, at " + uniqueKeyNode.getParserPosition()); - } + for (SqlNodeList uniqueKeys: this.uniqueKeysList) { + for (SqlNode uniqueKeyNode : uniqueKeys) { + String uniqueKey = ((SqlIdentifier) uniqueKeyNode).getSimple(); + if (!columnNames.contains(uniqueKey)) { + throw new SqlValidateException( + uniqueKeyNode.getParserPosition(), + "Unique key [" + uniqueKey + "] not defined in columns, at " + uniqueKeyNode.getParserPosition()); } } } - if (this.partitionKeyList != null) { - for (SqlNode partitionKeyNode : this.partitionKeyList.getList()) { - String partitionKey = ((SqlIdentifier) partitionKeyNode).getSimple(); - if (!columnNames.contains(partitionKey)) { - throw new SqlValidateException( - partitionKeyNode.getParserPosition(), - "Partition column [" + partitionKey + "] not defined in columns, at " - + partitionKeyNode.getParserPosition()); - } + for (SqlNode partitionKeyNode : this.partitionKeyList.getList()) { + String partitionKey = ((SqlIdentifier) partitionKeyNode).getSimple(); + if (!columnNames.contains(partitionKey)) { + throw new SqlValidateException( + partitionKeyNode.getParserPosition(), + "Partition column [" + partitionKey + "] not defined in columns, at " + + partitionKeyNode.getParserPosition()); } } @@ -254,14 +250,14 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { column.unparse(writer, leftPrec, rightPrec); } } - if (primaryKeyList != null && primaryKeyList.size() > 0) { + if (primaryKeyList.size() > 0) { printIndent(writer); writer.keyword("PRIMARY KEY"); SqlWriter.Frame keyFrame = writer.startList("(", ")"); primaryKeyList.unparse(writer, leftPrec, rightPrec); writer.endList(keyFrame); } - if (uniqueKeysList != null && uniqueKeysList.size() > 0) { + if (uniqueKeysList.size() > 0) { printIndent(writer); for (SqlNodeList uniqueKeyList : uniqueKeysList) { writer.keyword("UNIQUE"); @@ -280,7 +276,7 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { comment.unparse(writer, leftPrec, rightPrec); } - if (this.partitionKeyList != null && this.partitionKeyList.size() > 0) { + if (this.partitionKeyList.size() > 0) { writer.newlineAndIndent(); writer.keyword("PARTITIONED BY"); SqlWriter.Frame withFrame = writer.startList("(", ")"); @@ -289,7 +285,7 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { writer.newlineAndIndent(); } - if (propertyList != null) { + if (this.propertyList.size() > 0) { writer.keyword("WITH"); SqlWriter.Frame withFrame = writer.startList("(", ")"); for (SqlNode property : propertyList) { @@ -312,7 +308,7 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { */ public static class TableCreationContext { public List columnList = new ArrayList<>(); - public SqlNodeList primaryKeyList; + public SqlNodeList primaryKeyList = SqlNodeList.EMPTY; public List uniqueKeysList = new ArrayList<>(); } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateView.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateView.java index 0bb193b31c..868592a7de 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateView.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateView.java @@ -32,8 +32,13 @@ import org.apache.calcite.sql.SqlSpecialOperator; import org.apache.calcite.sql.SqlWriter; import org.apache.calcite.sql.parser.SqlParserPos; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.List; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; /** * CREATE VIEW DDL sql call. @@ -44,6 +49,8 @@ public class SqlCreateView extends SqlCreate implements ExtendedSqlNode { private final SqlIdentifier viewName; private final SqlNodeList fieldList; private final SqlNode query; + + @Nullable private final SqlCharStringLiteral comment; public SqlCreateView( @@ -54,9 +61,9 @@ public class SqlCreateView extends SqlCreate implements ExtendedSqlNode { boolean replace, SqlCharStringLiteral comment) { super(OPERATOR, pos, replace, false); - this.viewName = viewName; - this.fieldList = fieldList; - this.query = query; + this.viewName = requireNonNull(viewName, "viewName should not be null"); + this.fieldList = requireNonNull(fieldList, "fieldList should not be null"); + this.query = requireNonNull(query, "query should not be null"); this.comment = comment; } @@ -82,8 +89,8 @@ public class SqlCreateView extends SqlCreate implements ExtendedSqlNode { return query; } - public SqlCharStringLiteral getComment() { - return comment; + public Optional getComment() { + return Optional.ofNullable(comment); } @Override diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlTableColumn.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlTableColumn.java index 8a63683e27..83e7376f20 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlTableColumn.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlTableColumn.java @@ -30,7 +30,10 @@ import org.apache.calcite.sql.SqlWriter; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.util.ImmutableNullableList; +import javax.annotation.Nullable; + import java.util.List; +import java.util.Optional; import static java.util.Objects.requireNonNull; @@ -43,6 +46,8 @@ public class SqlTableColumn extends SqlCall { private SqlIdentifier name; private SqlDataTypeSpec type; + + @Nullable private SqlCharStringLiteral comment; public SqlTableColumn(SqlIdentifier name, @@ -96,8 +101,8 @@ public class SqlTableColumn extends SqlCall { this.type = type; } - public SqlCharStringLiteral getComment() { - return comment; + public Optional getComment() { + return Optional.ofNullable(comment); } public void setComment(SqlCharStringLiteral comment) { diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java index 36c742a2ad..c4d1dc4960 100644 --- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java @@ -687,6 +687,18 @@ public class FlinkSqlParserImplTest extends SqlParserTest { "CAST(`A` AS ROW(`F0` VARCHAR NOT NULL, `F1` TIMESTAMP) MULTISET)"); } + @Test + public void testCreateTableWithNakedTableName() { + String sql = "CREATE TABLE tbl1"; + sql(sql).node(new ValidationMatcher()); + } + + @Test + public void testCreateViewWithEmptyFields() { + String sql = "CREATE VIEW v1 AS SELECT 1"; + sql(sql).node(new ValidationMatcher()); + } + /** Matcher that invokes the #validate() of the {@link ExtendedSqlNode} instance. **/ private static class ValidationMatcher extends BaseMatcher { private String expectedColumnSql; diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java index d41ad70d5a..3a12965367 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java @@ -42,7 +42,6 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -99,37 +98,27 @@ public class SqlToOperationConverter { */ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) { // primary key and unique keys are not supported - if ((sqlCreateTable.getPrimaryKeyList() != null - && sqlCreateTable.getPrimaryKeyList().size() > 0) - || (sqlCreateTable.getUniqueKeysList() != null - && sqlCreateTable.getUniqueKeysList().size() > 0)) { + if ((sqlCreateTable.getPrimaryKeyList().size() > 0) + || (sqlCreateTable.getUniqueKeysList().size() > 0)) { throw new SqlConversionException("Primary key and unique key are not supported yet."); } // set with properties - SqlNodeList propertyList = sqlCreateTable.getPropertyList(); Map properties = new HashMap<>(); - if (propertyList != null) { - propertyList.getList().forEach(p -> - properties.put(((SqlTableOption) p).getKeyString().toLowerCase(), - ((SqlTableOption) p).getValueString())); - } + sqlCreateTable.getPropertyList().getList().forEach(p -> + properties.put(((SqlTableOption) p).getKeyString().toLowerCase(), + ((SqlTableOption) p).getValueString())); TableSchema tableSchema = createTableSchema(sqlCreateTable); - String tableComment = ""; - if (sqlCreateTable.getComment() != null) { - tableComment = sqlCreateTable.getComment().getNlsString().getValue(); - } + String tableComment = sqlCreateTable.getComment().map(comment -> + comment.getNlsString().getValue()).orElse(null); // set partition key - List partitionKeys = new ArrayList<>(); - SqlNodeList partitionKey = sqlCreateTable.getPartitionKeyList(); - if (partitionKey != null) { - partitionKeys = partitionKey - .getList() - .stream() - .map(p -> ((SqlIdentifier) p).getSimple()) - .collect(Collectors.toList()); - } + List partitionKeys = sqlCreateTable.getPartitionKeyList() + .getList() + .stream() + .map(p -> ((SqlIdentifier) p).getSimple()) + .collect(Collectors.toList()); + CatalogTable catalogTable = new CatalogTableImpl(tableSchema, partitionKeys, properties, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java index f44365f5f1..c7566602f7 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java @@ -43,7 +43,6 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -100,37 +99,27 @@ public class SqlToOperationConverter { */ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) { // primary key and unique keys are not supported - if ((sqlCreateTable.getPrimaryKeyList() != null - && sqlCreateTable.getPrimaryKeyList().size() > 0) - || (sqlCreateTable.getUniqueKeysList() != null - && sqlCreateTable.getUniqueKeysList().size() > 0)) { + if ((sqlCreateTable.getPrimaryKeyList().size() > 0) + || (sqlCreateTable.getUniqueKeysList().size() > 0)) { throw new SqlConversionException("Primary key and unique key are not supported yet."); } // set with properties - SqlNodeList propertyList = sqlCreateTable.getPropertyList(); Map properties = new HashMap<>(); - if (propertyList != null) { - propertyList.getList().forEach(p -> - properties.put(((SqlTableOption) p).getKeyString().toLowerCase(), - ((SqlTableOption) p).getValueString())); - } + sqlCreateTable.getPropertyList().getList().forEach(p -> + properties.put(((SqlTableOption) p).getKeyString().toLowerCase(), + ((SqlTableOption) p).getValueString())); TableSchema tableSchema = createTableSchema(sqlCreateTable); - String tableComment = ""; - if (sqlCreateTable.getComment() != null) { - tableComment = sqlCreateTable.getComment().getNlsString().getValue(); - } + String tableComment = sqlCreateTable.getComment().map(comment -> + comment.getNlsString().getValue()).orElse(null); // set partition key - List partitionKeys = new ArrayList<>(); - SqlNodeList partitionKey = sqlCreateTable.getPartitionKeyList(); - if (partitionKey != null) { - partitionKeys = partitionKey - .getList() - .stream() - .map(p -> ((SqlIdentifier) p).getSimple()) - .collect(Collectors.toList()); - } + List partitionKeys = sqlCreateTable.getPartitionKeyList() + .getList() + .stream() + .map(p -> ((SqlIdentifier) p).getSimple()) + .collect(Collectors.toList()); + CatalogTable catalogTable = new CatalogTableImpl(tableSchema, partitionKeys, properties, -- Gitee From 86ae90ed114779e27a542a59c523c2f0eadba3a3 Mon Sep 17 00:00:00 2001 From: fanrui <1996fanrui@gmail.com> Date: Wed, 16 Oct 2019 20:57:14 +0800 Subject: [PATCH 257/268] [hotfix][docs-zh] Fix the incorrect links in "Savepoints" page This closes #9911 --- docs/ops/state/savepoints.zh.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ops/state/savepoints.zh.md b/docs/ops/state/savepoints.zh.md index e601e7f129..6bdb9df74a 100644 --- a/docs/ops/state/savepoints.zh.md +++ b/docs/ops/state/savepoints.zh.md @@ -30,7 +30,7 @@ under the License. Savepoint 是依据 Flink [checkpointing 机制]({{ site.baseurl }}/zh/internals/stream_checkpointing.html)所创建的流作业执行状态的一致镜像。 你可以使用 Savepoint 进行 Flink 作业的停止与重启、fork 或者更新。 Savepoint 由两部分组成:稳定存储(列入 HDFS,S3,...) 上包含二进制文件的目录(通常很大),和元数据文件(相对较小)。 稳定存储上的文件表示作业执行状态的数据镜像。 Savepoint 的元数据文件以(绝对路径)的形式包含(主要)指向作为 Savepoint 一部分的稳定存储上的所有文件的指针。

    -注意: 为了允许程序和 Flink 版本之间的升级,请务必查看以下有关分配算子 ID 的部分 。 +注意: 为了允许程序和 Flink 版本之间的升级,请务必查看以下有关分配算子 ID 的部分 。
    从概念上讲, Flink 的 Savepoint 与 Checkpoint 的不同之处类似于传统数据库中的备份与恢复日志之间的差异。 Checkpoint 的主要目的是为意外失败的作业提供恢复机制。 Checkpoint 的生命周期由 Flink 管理,即 Flink 创建,管理和删除 Checkpoint - 无需用户交互。 作为一种恢复和定期触发的方法,Checkpoint 实现有两个设计目标:i)轻量级创建和 ii)尽可能快地恢复。 可能会利用某些特定的属性来达到这个,例如, 工作代码在执行尝试之间不会改变。 在用户终止作业后,通常会删除 Checkpoint(除非明确配置为保留的 Checkpoint)。 @@ -78,7 +78,7 @@ mapper-id | State of StatefulMapper ### 触发 Savepoint -当触发 Savepoint 时,将创建一个新的 Savepoint 目录,其中存储数据和元数据。可以通过[配置默认目标目录](#configuration)或使用触发器命令指定自定义目标目录(参见[`:targetDirectory`参数](#触发-savepoint-1)来控制该目录的位置。 +当触发 Savepoint 时,将创建一个新的 Savepoint 目录,其中存储数据和元数据。可以通过[配置默认目标目录](#配置)或使用触发器命令指定自定义目标目录(参见[`:targetDirectory`参数](#触发-savepoint-1)来控制该目录的位置。
    注意:目标目录必须是 JobManager(s) 和 TaskManager(s) 都可以访问的位置,例如分布式文件系统上的位置。 -- Gitee From 9e7f229be0ace6d7cd5196367d67162e041fdcb0 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Thu, 17 Oct 2019 14:39:20 +0800 Subject: [PATCH 258/268] [docs-sync] Synchronize the latest documentation changes (commits to 86ae90ed) into Chinese documents --- docs/dev/connectors/elasticsearch.zh.md | 8 +- docs/dev/connectors/kinesis.zh.md | 67 ++++- docs/dev/table/connect.zh.md | 247 +++++++++++++++++- docs/dev/table/sourceSinks.zh.md | 2 +- .../dev/table/streaming/match_recognize.zh.md | 9 +- docs/dev/table/types.zh.md | 38 +++ docs/ops/config.zh.md | 13 +- 7 files changed, 361 insertions(+), 23 deletions(-) diff --git a/docs/dev/connectors/elasticsearch.zh.md b/docs/dev/connectors/elasticsearch.zh.md index e816da873e..915860fbd2 100644 --- a/docs/dev/connectors/elasticsearch.zh.md +++ b/docs/dev/connectors/elasticsearch.zh.md @@ -53,7 +53,7 @@ of the Elasticsearch installation: flink-connector-elasticsearch6{{ site.scala_version_suffix }} 1.6.0 - 6 and later versions + 6.x flink-connector-elasticsearch7{{ site.scala_version_suffix }} @@ -128,7 +128,7 @@ input.addSink(new ElasticsearchSink<>(config, transportAddresses, new Elasticsea } }));{% endhighlight %}
    -
    +
    {% highlight java %} import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.streaming.api.datastream.DataStream; @@ -231,7 +231,7 @@ input.addSink(new ElasticsearchSink(config, transportAddresses, new Elasticsearc })) {% endhighlight %}
    -
    +
    {% highlight scala %} import org.apache.flink.api.common.functions.RuntimeContext import org.apache.flink.streaming.api.datastream.DataStream @@ -252,7 +252,7 @@ val httpHosts = new java.util.ArrayList[HttpHost] httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")) httpHosts.add(new HttpHost("10.2.3.1", 9200, "http")) -val esSinkBuilder = new ElasticsearchSink.Builer[String]( +val esSinkBuilder = new ElasticsearchSink.Builder[String]( httpHosts, new ElasticsearchSinkFunction[String] { def createIndexRequest(element: String): IndexRequest = { diff --git a/docs/dev/connectors/kinesis.zh.md b/docs/dev/connectors/kinesis.zh.md index 4d7ceea6d8..48849c912c 100644 --- a/docs/dev/connectors/kinesis.zh.md +++ b/docs/dev/connectors/kinesis.zh.md @@ -38,7 +38,6 @@ To use the connector, add the following Maven dependency to your project: {% endhighlight %} - Attention Prior to Flink version 1.10.0 the `flink-connector-kinesis{{ site.scala_version_suffix }}` has a dependency on code licensed under the [Amazon Software License](https://aws.amazon.com/asl/). Linking to the prior versions of flink-connector-kinesis will include this code into your application. @@ -187,20 +186,72 @@ it can be passed to the consumer in the following way:
    {% highlight java %} -DataStream kinesis = env.addSource(new FlinkKinesisConsumer<>( - "kinesis_stream_name", new SimpleStringSchema(), kinesisConsumerConfig)); -kinesis = kinesis.assignTimestampsAndWatermarks(new CustomTimestampAssigner()); +FlinkKinesisConsumer consumer = new FlinkKinesisConsumer<>( + "kinesis_stream_name", + new SimpleStringSchema(), + kinesisConsumerConfig); +consumer.setPeriodicWatermarkAssigner(new CustomAssignerWithPeriodicWatermarks()); +DataStream stream = env + .addSource(consumer) + .print(); {% endhighlight %}
    {% highlight scala %} -val kinesis = env.addSource(new FlinkKinesisConsumer[String]( - "kinesis_stream_name", new SimpleStringSchema, kinesisConsumerConfig)) -kinesis = kinesis.assignTimestampsAndWatermarks(new CustomTimestampAssigner) +val consumer = new FlinkKinesisConsumer[String]( + "kinesis_stream_name", + new SimpleStringSchema(), + kinesisConsumerConfig); +consumer.setPeriodicWatermarkAssigner(new CustomAssignerWithPeriodicWatermarks()); +val stream = env + .addSource(consumer) + .print(); {% endhighlight %}
    +Internally, an instance of the assigner is executed per shard / consumer thread (see threading model below). +When an assigner is specified, for each record read from Kinesis, the extractTimestamp(T element, long previousElementTimestamp) +is called to assign a timestamp to the record and getCurrentWatermark() to determine the new watermark for the shard. +The watermark of the consumer subtask is then determined as the minimum watermark of all its shards and emitted periodically. +The per shard watermark is essential to deal with varying consumption speed between shards, that otherwise could lead +to issues with downstream logic that relies on the watermark, such as incorrect late data dropping. + +By default, the watermark is going to stall if shards do not deliver new records. +The property `ConsumerConfigConstants.SHARD_IDLE_INTERVAL_MILLIS` can be used to avoid this potential issue through a +timeout that will allow the watermark to progress despite of idle shards. + +### Event Time Alignment for Shard Consumers + +The Flink Kinesis Consumer optionally supports synchronization between parallel consumer subtasks (and their threads) +to avoid the event time skew related problems described in [Event time synchronization across sources](https://issues.apache.org/jira/browse/FLINK-10886). + +To enable synchronization, set the watermark tracker on the consumer: + +
    +{% highlight java %} +JobManagerWatermarkTracker watermarkTracker = + new JobManagerWatermarkTracker("myKinesisSource"); +consumer.setWatermarkTracker(watermarkTracker); +{% endhighlight %} +
    + +The `JobManagerWatermarkTracker` will use a global aggregate to synchronize the per subtask watermarks. Each subtask +uses a per shard queue to control the rate at which records are emitted downstream based on how far ahead of the global +watermark the next record in the queue is. + +The "emit ahead" limit is configured via `ConsumerConfigConstants.WATERMARK_LOOKAHEAD_MILLIS`. Smaller values reduce +the skew but also the throughput. Larger values will allow the subtask to proceed further before waiting for the global +watermark to advance. + +Another variable in the throughput equation is how frequently the watermark is propagated by the tracker. +The interval can be configured via `ConsumerConfigConstants.WATERMARK_SYNC_MILLIS`. +Smaller values reduce emitter waits and come at the cost of increased communication with the job manager. + +Since records accumulate in the queues when skew occurs, increased memory consumption needs to be expected. +How much depends on the average record size. With larger sizes, it may be necessary to adjust the emitter queue capacity via +`ConsumerConfigConstants.WATERMARK_SYNC_QUEUE_CAPACITY`. + ### Threading Model The Flink Kinesis Consumer uses multiple threads for shard discovery and data consumption. @@ -222,7 +273,7 @@ on the APIs, the consumer will be competing with other non-Flink consuming appli Below is a list of APIs called by the consumer with description of how the consumer uses the API, as well as information on how to deal with any errors or warnings that the Flink Kinesis Consumer may have due to these service limits. -- *[DescribeStream](http://docs.aws.amazon.com/kinesis/latest/APIReference/API_DescribeStream.html)*: this is constantly called +- *[ListShards](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ListShards.html)*: this is constantly called by a single thread in each parallel consumer subtask to discover any new shards as a result of stream resharding. By default, the consumer performs the shard discovery at an interval of 10 seconds, and will retry indefinitely until it gets a result from Kinesis. If this interferes with other non-Flink consuming applications, users can slow down the consumer of diff --git a/docs/dev/table/connect.zh.md b/docs/dev/table/connect.zh.md index 9040cf1cba..130f55ebf9 100644 --- a/docs/dev/table/connect.zh.md +++ b/docs/dev/table/connect.zh.md @@ -49,6 +49,8 @@ The following tables list all available connectors and formats. Their mutual com | Apache Kafka | 0.10 | `flink-connector-kafka-0.10` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-kafka-0.10{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-kafka-0.10{{site.scala_version_suffix}}-{{site.version}}.jar) | | Apache Kafka | 0.11 | `flink-connector-kafka-0.11` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-kafka-0.11{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-kafka-0.11{{site.scala_version_suffix}}-{{site.version}}.jar) | | Apache Kafka | 0.11+ (`universal`) | `flink-connector-kafka` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-kafka{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-kafka{{site.scala_version_suffix}}-{{site.version}}.jar) | +| HBase | 1.4.3 | `flink-hbase` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-hbase{{site.scala_version_suffix}}/{{site.version}}/flink-hbase{{site.scala_version_suffix}}-{{site.version}}.jar) | +| JDBC | | `flink-jdbc` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-jdbc{{site.scala_version_suffix}}/{{site.version}}/flink-jdbc{{site.scala_version_suffix}}-{{site.version}}.jar) | ### Formats @@ -225,7 +227,7 @@ table_environment \ .field("message", DataTypes.STRING()) ) \ .in_append_mode() \ - .register_table_source("MyUserTable") + .register_table_source("MyUserTable") # specify the update-mode for streaming tables and # register as source, sink, or both and under a name {% endhighlight %} @@ -1075,6 +1077,245 @@ CREATE TABLE MyUserTable ( {% top %} +### HBase Connector + +Source: Batch +Sink: Batch +Sink: Streaming Append Mode +Sink: Streaming Upsert Mode +Temporal Join: Sync Mode + +The HBase connector allows for reading from and writing to an HBase cluster. + +The connector can operate in [upsert mode](#update-modes) for exchanging UPSERT/DELETE messages with the external system using a [key defined by the query](./streaming/dynamic_tables.html#table-to-stream-conversion). + +For append-only queries, the connector can also operate in [append mode](#update-modes) for exchanging only INSERT messages with the external system. + +The connector can be defined as follows: + +
    +
    +{% highlight java %} +.connect( + new HBase() + .version("1.4.3") // required: currently only support "1.4.3" + .tableName("hbase_table_name") // required: HBase table name + .zookeeperQuorum("localhost:2181") // required: HBase Zookeeper quorum configuration + .zookeeperNodeParent("/test") // optional: the root dir in Zookeeper for HBase cluster. + // The default value is "/hbase". + .writeBufferFlushMaxSize("10mb") // optional: writing option, determines how many size in memory of buffered + // rows to insert per round trip. This can help performance on writing to JDBC + // database. The default value is "2mb". + .writeBufferFlushMaxRows(1000) // optional: writing option, determines how many rows to insert per round trip. + // This can help performance on writing to JDBC database. No default value, + // i.e. the default flushing is not depends on the number of buffered rows. + .writeBufferFlushInterval("2s") // optional: writing option, sets a flush interval flushing buffered requesting + // if the interval passes, in milliseconds. Default value is "0s", which means + // no asynchronous flush thread will be scheduled. +) +{% endhighlight %} +
    +
    +{% highlight yaml %} +connector: + type: hbase + version: "1.4.3" # required: currently only support "1.4.3" + + table-name: "hbase_table_name" # required: HBase table name + + zookeeper: + quorum: "localhost:2181" # required: HBase Zookeeper quorum configuration + znode.parent: "/test" # optional: the root dir in Zookeeper for HBase cluster. + # The default value is "/hbase". + + write.buffer-flush: + max-size: "10mb" # optional: writing option, determines how many size in memory of buffered + # rows to insert per round trip. This can help performance on writing to JDBC + # database. The default value is "2mb". + max-rows: 1000 # optional: writing option, determines how many rows to insert per round trip. + # This can help performance on writing to JDBC database. No default value, + # i.e. the default flushing is not depends on the number of buffered rows. + interval: "2s" # optional: writing option, sets a flush interval flushing buffered requesting + # if the interval passes, in milliseconds. Default value is "0s", which means + # no asynchronous flush thread will be scheduled. +{% endhighlight %} +
    + +
    +{% highlight sql %} +CREATE TABLE MyUserTable ( + hbase_rowkey_name rowkey_type, + hbase_column_family_name1 ROW<...>, + hbase_column_family_name2 ROW<...> +) WITH ( + 'connector.type' = 'hbase', -- required: specify this table type is hbase + + 'connector.version' = '1.4.3', -- required: valid connector versions are "1.4.3" + + 'connector.table-name' = 'hbase_table_name', -- required: hbase table name + + 'connector.zookeeper.quorum' = 'localhost:2181', -- required: HBase Zookeeper quorum configuration + 'connector.zookeeper.znode.parent' = '/test', -- optional: the root dir in Zookeeper for HBase cluster. + -- The default value is "/hbase". + + 'connector.write.buffer-flush.max-size' = '10mb', -- optional: writing option, determines how many size in memory of buffered + -- rows to insert per round trip. This can help performance on writing to JDBC + -- database. The default value is "2mb". + + 'connector.write.buffer-flush.max-rows' = '1000', -- optional: writing option, determines how many rows to insert per round trip. + -- This can help performance on writing to JDBC database. No default value, + -- i.e. the default flushing is not depends on the number of buffered rows. + + 'connector.write.buffer-flush.interval' = '2s', -- optional: writing option, sets a flush interval flushing buffered requesting + -- if the interval passes, in milliseconds. Default value is "0s", which means + -- no asynchronous flush thread will be scheduled. +) +{% endhighlight %} +
    +
    + +**Columns:** All the column families in HBase table must be declared as `ROW` type, the field name maps to the column family name, and the nested field names map to the column qualifier names. There is no need to declare all the families and qualifiers in the schema, users can declare what's necessary. Except the `ROW` type fields, the only one field of atomic type (e.g. `STRING`, `BIGINT`) will be recognized as row key of the table. There's no constraints on the name of row key field. + +**Temporary join:** Lookup join against HBase do not use any caching; data is always queired directly through the HBase client. + +{% top %} + +### JDBC Connector + +Source: Batch +Sink: Batch +Sink: Streaming Append Mode +Sink: Streaming Upsert Mode +Temporal Join: Sync Mode + +The JDBC connector allows for reading from and writing into an JDBC client. + +The connector can operate in [upsert mode](#update-modes) for exchanging UPSERT/DELETE messages with the external system using a [key defined by the query](./streaming/dynamic_tables.html#table-to-stream-conversion). + +For append-only queries, the connector can also operate in [append mode](#update-modes) for exchanging only INSERT messages with the external system. + +To use JDBC connector, need to choose an actual driver to use. Here are drivers currently supported: + +**Supported Drivers:** + +| Name | Group Id | Artifact Id | JAR | +| :-----------| :------------------| :--------------------| :----------------| +| MySQL | mysql | mysql-connector-java | [Download](http://central.maven.org/maven2/mysql/mysql-connector-java/) | +| PostgreSQL | org.postgresql | postgresql | [Download](https://jdbc.postgresql.org/download.html) | +| Derby | org.apache.derby | derby | [Download](http://db.apache.org/derby/derby_downloads.html) | + +
    + +The connector can be defined as follows: + +
    +
    +{% highlight yaml %} +connector: + type: jdbc + url: "jdbc:mysql://localhost:3306/flink-test" # required: JDBC DB url + table: "jdbc_table_name" # required: jdbc table name + driver: "com.mysql.jdbc.Driver" # optional: the class name of the JDBC driver to use to connect to this URL. + # If not set, it will automatically be derived from the URL. + + username: "name" # optional: jdbc user name and password + password: "password" + + read: # scan options, optional, used when reading from table + partition: # These options must all be specified if any of them is specified. In addition, partition.num must be specified. They + # describe how to partition the table when reading in parallel from multiple tasks. partition.column must be a numeric, + # date, or timestamp column from the table in question. Notice that lowerBound and upperBound are just used to decide + # the partition stride, not for filtering the rows in table. So all rows in the table will be partitioned and returned. + # This option applies only to reading. + column: "column_name" # optional, name of the column used for partitioning the input. + num: 50 # optional, the number of partitions. + lower-bound: 500 # optional, the smallest value of the first partition. + upper-bound: 1000 # optional, the largest value of the last partition. + fetch-size: 100 # optional, Gives the reader a hint as to the number of rows that should be fetched + # from the database when reading per round trip. If the value specified is zero, then + # the hint is ignored. The default value is zero. + + lookup: # lookup options, optional, used in temporary join + cache: + max-rows: 5000 # optional, max number of rows of lookup cache, over this value, the oldest rows will + # be eliminated. "cache.max-rows" and "cache.ttl" options must all be specified if any + # of them is specified. Cache is not enabled as default. + ttl: "10s" # optional, the max time to live for each rows in lookup cache, over this time, the oldest rows + # will be expired. "cache.max-rows" and "cache.ttl" options must all be specified if any of + # them is specified. Cache is not enabled as default. + max-retries: 3 # optional, max retry times if lookup database failed + + write: # sink options, optional, used when writing into table + flush: + max-rows: 5000 # optional, flush max size (includes all append, upsert and delete records), + # over this number of records, will flush data. The default value is "5000". + interval: "2s" # optional, flush interval mills, over this time, asynchronous threads will flush data. + # The default value is "0s", which means no asynchronous flush thread will be scheduled. + max-retries: 3 # optional, max retry times if writing records to database failed. +{% endhighlight %} +
    + +
    +{% highlight sql %} +CREATE TABLE MyUserTable ( + ... +) WITH ( + 'connector.type' = 'jdbc', -- required: specify this table type is jdbc + + 'connector.url' = 'jdbc:mysql://localhost:3306/flink-test', -- required: JDBC DB url + + 'connector.table' = 'jdbc_table_name', -- required: jdbc table name + + 'connector.driver' = 'com.mysql.jdbc.Driver', -- optional: the class name of the JDBC driver to use to connect to this URL. + -- If not set, it will automatically be derived from the URL. + + 'connector.username' = 'name', -- optional: jdbc user name and password + 'connector.password' = 'password', + + -- scan options, optional, used when reading from table + + -- These options must all be specified if any of them is specified. In addition, partition.num must be specified. They + -- describe how to partition the table when reading in parallel from multiple tasks. partition.column must be a numeric, + -- date, or timestamp column from the table in question. Notice that lowerBound and upperBound are just used to decide + -- the partition stride, not for filtering the rows in table. So all rows in the table will be partitioned and returned. + -- This option applies only to reading. + 'connector.read.partition.column' = 'column_name', -- optional, name of the column used for partitioning the input. + 'connector.read.partition.num' = '50', -- optional, the number of partitions. + 'connector.read.partition.lower-bound' = '500', -- optional, the smallest value of the first partition. + 'connector.read.partition.upper-bound' = '1000', -- optional, the largest value of the last partition. + + 'connector.read.fetch-size' = '100', -- optional, Gives the reader a hint as to the number of rows that should be fetched + -- from the database when reading per round trip. If the value specified is zero, then + -- the hint is ignored. The default value is zero. + + -- lookup options, optional, used in temporary join + 'connector.lookup.cache.max-rows' = '5000', -- optional, max number of rows of lookup cache, over this value, the oldest rows will + -- be eliminated. "cache.max-rows" and "cache.ttl" options must all be specified if any + -- of them is specified. Cache is not enabled as default. + 'connector.lookup.cache.ttl' = '10s', -- optional, the max time to live for each rows in lookup cache, over this time, the oldest rows + -- will be expired. "cache.max-rows" and "cache.ttl" options must all be specified if any of + -- them is specified. Cache is not enabled as default. + 'connector.lookup.max-retries' = '3', -- optional, max retry times if lookup database failed + + -- sink options, optional, used when writing into table + 'connector.write.flush.max-rows' = '5000', -- optional, flush max size (includes all append, upsert and delete records), + -- over this number of records, will flush data. The default value is "5000". + 'connector.write.flush.interval' = '2s', -- optional, flush interval mills, over this time, asynchronous threads will flush data. + -- The default value is "0s", which means no asynchronous flush thread will be scheduled. + 'connector.write.max-retries' = '3' -- optional, max retry times if writing records to database failed +) +{% endhighlight %} +
    +
    + +**Upsert sink:** 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`. If a JDBC table is used as upsert sink, please make sure keys of the query is one of the unique key sets or primary key of the underlying database. This can guarantee the output result is as expected. + +**Temporary Join:** JDBC connector can be used in temporal join as a lookup source. Currently, only sync lookup mode is supported. The lookup cache options (`connector.lookup.cache.max-rows` and `connector.lookup.cache.ttl`) must all be specified if any of them is specified. The lookup cache is used to improve performance of temporal join JDBC connector by querying the cache first instead of send all requests to remote database. But the returned value might not be the latest if it is from the cache. So it's a balance between throughput and correctness. + +**Writing:** As default, the `connector.write.flush.interval` is `0s` and `connector.write.flush.max-rows` is `5000`, which means for low traffic queries, the buffered output rows may not be flushed to database for a long time. So the interval configuration is recommended to set. + +{% top %} + Table Formats ------------- @@ -1143,8 +1384,8 @@ The CSV format can be used as follows: .field_delimiter(';') # optional: field delimiter character (',' by default) .line_delimiter("\r\n") # optional: line delimiter ("\n" by default; # otherwise "\r", "\r\n", or "" are allowed) - .quote_character("'") # optional: quote character for enclosing field values ('"' by default) - .allow_comments() # optional: ignores comment lines that start with "#" (disabled by default); + .quote_character('\'') # optional: quote character for enclosing field values ('"' by default) + .allow_comments() # optional: ignores comment lines that start with '#' (disabled by default); # if enabled, make sure to also ignore parse errors to allow empty rows .ignore_parse_errors() # optional: skip fields and rows with parse errors instead of failing; # fields are set to null in case of errors diff --git a/docs/dev/table/sourceSinks.zh.md b/docs/dev/table/sourceSinks.zh.md index e37e7be613..7fd872832d 100644 --- a/docs/dev/table/sourceSinks.zh.md +++ b/docs/dev/table/sourceSinks.zh.md @@ -583,7 +583,7 @@ trait TableFactory {
    * `requiredContext()`: Specifies the context that this factory has been implemented for. The framework guarantees to only match for this factory if the specified set of properties and values are met. Typical properties might be `connector.type`, `format.type`, or `update-mode`. Property keys such as `connector.property-version` and `format.property-version` are reserved for future backwards compatibility cases. -* `supportedProperties`: List of property keys that this factory can handle. This method will be used for validation. If a property is passed that this factory cannot handle, an exception will be thrown. The list must not contain the keys that are specified by the context. +* `supportedProperties()`: List of property keys that this factory can handle. This method will be used for validation. If a property is passed that this factory cannot handle, an exception will be thrown. The list must not contain the keys that are specified by the context. In order to create a specific instance, a factory class can implement one or more interfaces provided in `org.apache.flink.table.factories`: diff --git a/docs/dev/table/streaming/match_recognize.zh.md b/docs/dev/table/streaming/match_recognize.zh.md index 4aeb110f6d..663c7d2a59 100644 --- a/docs/dev/table/streaming/match_recognize.zh.md +++ b/docs/dev/table/streaming/match_recognize.zh.md @@ -285,7 +285,7 @@ FROM Ticker LAST(A.rowtime) AS end_tstamp, AVG(A.price) AS avgPrice ONE ROW PER MATCH - AFTER MATCH SKIP TO FIRST B + AFTER MATCH SKIP PAST LAST ROW PATTERN (A+ B) DEFINE A AS AVG(A.price) < 15 @@ -307,19 +307,20 @@ symbol rowtime price tax 'ACME' '01-Apr-11 10:00:07' 10 2 'ACME' '01-Apr-11 10:00:08' 15 2 'ACME' '01-Apr-11 10:00:09' 25 2 -'ACME' '01-Apr-11 10:00:10' 30 1 +'ACME' '01-Apr-11 10:00:10' 25 1 +'ACME' '01-Apr-11 10:00:11' 30 1 {% endhighlight %} The query will accumulate events as part of the pattern variable `A` as long as the average price of them does not exceed `15`. For example, such a limit exceeding happens at `01-Apr-11 10:00:04`. -The following period exceeds the average price of `15` again at `01-Apr-11 10:00:10`. Thus the +The following period exceeds the average price of `15` again at `01-Apr-11 10:00:11`. Thus the results for said query will be: {% highlight text %} symbol start_tstamp end_tstamp avgPrice ========= ================== ================== ============ ACME 01-APR-11 10:00:00 01-APR-11 10:00:03 14.5 -ACME 01-APR-11 10:00:04 01-APR-11 10:00:09 13.5 +ACME 01-APR-11 10:00:05 01-APR-11 10:00:10 13.5 {% endhighlight %} Note Aggregations can be applied to expressions, but only if diff --git a/docs/dev/table/types.zh.md b/docs/dev/table/types.zh.md index 13139ff9a1..8a913bcfd6 100644 --- a/docs/dev/table/types.zh.md +++ b/docs/dev/table/types.zh.md @@ -1005,6 +1005,43 @@ equivalent to `ARRAY`. |:----------|:-----:|:------:|:----------------------------------| |*t*`[]` | (X) | (X) | Depends on the subtype. *Default* | +#### `MAP` + +Data type of an associative array that maps keys (including `NULL`) to values (including `NULL`). A map +cannot contain duplicate keys; each key can map to at most one value. + +There is no restriction of element types; it is the responsibility of the user to ensure uniqueness. + +The map type is an extension to the SQL standard. + +**Declaration** + +
    + +
    +{% highlight text %} +MAP +{% endhighlight %} +
    + +
    +{% highlight java %} +DataTypes.MAP(kt, vt) +{% endhighlight %} +
    + +
    + +The type can be declared using `MAP` where `kt` is the data type of the key elements +and `vt` is the data type of the value elements. + +**Bridging to JVM Types** + +| Java Type | Input | Output | Remarks | +|:--------------------------------------|:-----:|:------:|:----------| +| `java.util.Map` | X | X | *Default* | +| *subclass* of `java.util.Map` | X | | | + #### `MULTISET` Data type of a multiset (=bag). Unlike a set, it allows for multiple instances for each of its @@ -1042,6 +1079,7 @@ equivalent to `MULTISET`. | Java Type | Input | Output | Remarks | |:-------------------------------------|:-----:|:------:|:---------------------------------------------------------| |`java.util.Map` | X | X | Assigns each value to an integer multiplicity. *Default* | +| *subclass* of `java.util.Map` | X | | Assigns each value to an integer multiplicity. | #### `ROW` diff --git a/docs/ops/config.zh.md b/docs/ops/config.zh.md index dd497847ce..cfbd9cc7c1 100644 --- a/docs/ops/config.zh.md +++ b/docs/ops/config.zh.md @@ -209,10 +209,10 @@ unless user define a `OptionsFactory` and set via `RocksDBStateBackend.setOption ### RocksDB Native Metrics Certain RocksDB native metrics may be forwarded to Flink's metrics reporter. -All native metrics are scoped to operators and then further broken down by column family; values are reported as unsigned longs. +All native metrics are scoped to operators and then further broken down by column family; values are reported as unsigned longs.
    - Note: Enabling native metrics may cause degraded performance and should be set carefully. + Note: Enabling native metrics may cause degraded performance and should be set carefully.
    {% include generated/rocks_db_native_metric_configuration.html %} @@ -231,7 +231,6 @@ You have to configure `jobmanager.archive.fs.dir` in order to archive terminated ## Background - ### Configuring the Network Buffers If you ever see the Exception `java.io.IOException: Insufficient number of network buffers`, you @@ -319,4 +318,12 @@ When starting a Flink application, users can supply the default number of slots +### Configuration Runtime Environment Variables +You have to set config with prefix `containerized.master.env.` and `containerized.taskmanager.env.` in order to set redefined environment variable in ApplicationMaster and TaskManager. + +- `containerized.master.env.`: Prefix for passing custom environment variables to Flink's master process. + For example for passing LD_LIBRARY_PATH as an env variable to the AppMaster, set containerized.master.env.LD_LIBRARY_PATH: "/usr/lib/native" + in the flink-conf.yaml. +- `containerized.taskmanager.env.`: Similar to the above, this configuration prefix allows setting custom environment variables for the workers (TaskManagers). + {% top %} -- Gitee From 60b5698aa2b9fe9ae233957688bd67a63325b7db Mon Sep 17 00:00:00 2001 From: lining Date: Thu, 17 Oct 2019 17:03:11 +0800 Subject: [PATCH 259/268] [FLINK-14176][web] Add log url for taskmanager of vertex (#9798) --- .../src/app/interfaces/job-vertex-task-manager.ts | 1 + .../job-overview-drawer-taskmanagers.component.html | 9 ++++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/flink-runtime-web/web-dashboard/src/app/interfaces/job-vertex-task-manager.ts b/flink-runtime-web/web-dashboard/src/app/interfaces/job-vertex-task-manager.ts index 822e8373a6..91f7e98615 100644 --- a/flink-runtime-web/web-dashboard/src/app/interfaces/job-vertex-task-manager.ts +++ b/flink-runtime-web/web-dashboard/src/app/interfaces/job-vertex-task-manager.ts @@ -50,4 +50,5 @@ export interface VertexTaskManagerDetailInterface { RUNNING: number; SCHEDULED: number; }; + 'taskmanager-id': string; } diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/taskmanagers/job-overview-drawer-taskmanagers.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/taskmanagers/job-overview-drawer-taskmanagers.component.html index a3fc5bbc86..2345c85fd6 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/taskmanagers/job-overview-drawer-taskmanagers.component.html +++ b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/taskmanagers/job-overview-drawer-taskmanagers.component.html @@ -21,12 +21,13 @@ [nzSize]="'small'" [nzLoading]="isLoading" [nzData]="listOfTaskManager" - [nzScroll]="{x:'1450px',y:'calc( 100% - 35px )'}" + [nzScroll]="{x:'1500px',y:'calc( 100% - 35px )'}" [nzFrontPagination]="false" [nzShowPagination]="false"> Host + LOG Bytes received Records received Bytes sent @@ -41,6 +42,12 @@ {{ taskManager.host }} + + - + + LOG + + {{ taskManager.metrics['read-bytes'] | humanizeBytes }} -- Gitee From 3f75754e2e506f7db54719ac2d605e365f1b5514 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 17 Oct 2019 13:04:29 +0200 Subject: [PATCH 260/268] [FLINK-14413][build] Specify encoding for ApacheNoticeResourceTransformer --- flink-dist/pom.xml | 1 + pom.xml | 1 + 2 files changed, 2 insertions(+) diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index 5a27808319..c112bc6c52 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -581,6 +581,7 @@ under the License. Apache Flink + UTF-8 diff --git a/pom.xml b/pom.xml index 08978f9746..7eab0bdb76 100644 --- a/pom.xml +++ b/pom.xml @@ -1589,6 +1589,7 @@ under the License. Apache Flink + UTF-8 -- Gitee From 62aa48795936f9700c38266a8a57b24ac46eaef6 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 9 Oct 2019 10:18:48 +0200 Subject: [PATCH 261/268] [hotfix][release] Use release version in path --- tools/releasing/create_binary_release.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/releasing/create_binary_release.sh b/tools/releasing/create_binary_release.sh index 60a3cca85d..8a1aa6d50e 100755 --- a/tools/releasing/create_binary_release.sh +++ b/tools/releasing/create_binary_release.sh @@ -74,7 +74,7 @@ make_binary_release() { # enable release profile here (to check for the maven version) $MVN clean package $FLAGS -Prelease -pl flink-dist -am -Dgpg.skip -Dcheckstyle.skip=true -DskipTests - cd flink-dist/target/flink-*-bin/ + cd flink-dist/target/flink-${RELEASE_VERSION}-bin tar czf "${dir_name}.tgz" flink-* cp flink-*.tgz ${RELEASE_DIR} -- Gitee From 8046f046e307164e62b662a30ef617a1eb99e259 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 9 Oct 2019 10:20:57 +0200 Subject: [PATCH 262/268] [FLINK-14008][release] Generate binary licensing during release --- tools/releasing/collect_license_files.sh | 5 ++--- tools/releasing/create_binary_release.sh | 1 + 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tools/releasing/collect_license_files.sh b/tools/releasing/collect_license_files.sh index 5aa7585410..754b5e8043 100755 --- a/tools/releasing/collect_license_files.sh +++ b/tools/releasing/collect_license_files.sh @@ -49,15 +49,14 @@ do (cd "${DIR}" && jar xf ${JAR} META-INF/NOTICE META-INF/licenses) done -NOTICE="${DST}/NOTICE-binary" +NOTICE="${DST}/NOTICE" [ -f "${NOTICE}" ] && rm "${NOTICE}" cp "${NOTICE_BINARY_PREAMBLE}" "${NOTICE}" (export LC_ALL=C; find "${TMP}" -name "NOTICE" | sort | xargs cat >> "${NOTICE}") -LICENSES="${DST}/licenses-binary" +LICENSES="${DST}/licenses" [ -f "${LICENSES}" ] && rm -r "${LICENSES}" find "${TMP}" -name "licenses" -type d -exec cp -r -- "{}" "${DST}" \; -mv "${DST}/licenses" "${LICENSES}" cp "${SLF4J_LICENSE}" "${LICENSES}" rm -r "${TMP}" diff --git a/tools/releasing/create_binary_release.sh b/tools/releasing/create_binary_release.sh index 8a1aa6d50e..6630106638 100755 --- a/tools/releasing/create_binary_release.sh +++ b/tools/releasing/create_binary_release.sh @@ -75,6 +75,7 @@ make_binary_release() { $MVN clean package $FLAGS -Prelease -pl flink-dist -am -Dgpg.skip -Dcheckstyle.skip=true -DskipTests cd flink-dist/target/flink-${RELEASE_VERSION}-bin + ${FLINK_DIR}/tools/releasing/collect_license_files.sh ./flink-${RELEASE_VERSION} ./flink-${RELEASE_VERSION} tar czf "${dir_name}.tgz" flink-* cp flink-*.tgz ${RELEASE_DIR} -- Gitee From a37a547a8f7f07c036d6b86ad93254b7d67280f7 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 9 Oct 2019 10:21:28 +0200 Subject: [PATCH 263/268] [FLINK-14008][release] Remove redundant NOTICE-binary infrastructure --- NOTICE-binary | 16732 ---------------- flink-dist/src/main/assemblies/bin.xml | 12 - 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 | 21 - 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.ant-design-palettes | 22 - licenses-binary/LICENSE.asm | 31 - licenses-binary/LICENSE.base64 | 26 - licenses-binary/LICENSE.bootstrap | 21 - licenses-binary/LICENSE.cloudpickle | 32 - licenses-binary/LICENSE.core-js | 21 - 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 - .../LICENSE.google-auth-library-credentials | 28 - licenses-binary/LICENSE.graphlib | 19 - licenses-binary/LICENSE.grizzled-slf4j | 11 - licenses-binary/LICENSE.imagesloaded | 7 - licenses-binary/LICENSE.influx | 21 - licenses-binary/LICENSE.janino | 31 - licenses-binary/LICENSE.jline | 7 - licenses-binary/LICENSE.jquery | 20 - licenses-binary/LICENSE.jsr166y | 26 - licenses-binary/LICENSE.jzlib | 26 - licenses-binary/LICENSE.kryo | 10 - licenses-binary/LICENSE.lodash | 10 - licenses-binary/LICENSE.minlog | 10 - licenses-binary/LICENSE.moment | 22 - .../LICENSE.moment-duration-format | 21 - licenses-binary/LICENSE.monaco-editor | 9 - licenses-binary/LICENSE.ng-zorro-antd | 22 - licenses-binary/LICENSE.protobuf | 32 - licenses-binary/LICENSE.py4j | 26 - licenses-binary/LICENSE.pyrolite | 21 - licenses-binary/LICENSE.qtip2 | 22 - licenses-binary/LICENSE.rxjs | 201 - licenses-binary/LICENSE.scala | 11 - licenses-binary/LICENSE.scopt | 21 - licenses-binary/LICENSE.slf4j | 21 - licenses-binary/LICENSE.slf4j-api | 21 - licenses-binary/LICENSE.split | 32 - licenses-binary/LICENSE.tinycolor2 | 21 - licenses-binary/LICENSE.tslib | 57 - licenses-binary/LICENSE.webbit | 38 - licenses-binary/LICENSE.zone | 21 - tools/travis_controller.sh | 33 - 55 files changed, 18294 deletions(-) delete mode 100644 NOTICE-binary delete mode 100644 licenses-binary/LICENSE-hdrhistogram delete mode 100644 licenses-binary/LICENSE-protobuf delete mode 100644 licenses-binary/LICENSE-re2j delete mode 100644 licenses-binary/LICENSE-stax2api delete mode 100644 licenses-binary/LICENSE-xmlenc delete mode 100644 licenses-binary/LICENSE.@angular delete mode 100644 licenses-binary/LICENSE.angular delete mode 100644 licenses-binary/LICENSE.angular-drag-and-drop-list delete mode 100644 licenses-binary/LICENSE.angular-moment delete mode 100644 licenses-binary/LICENSE.angular-ui-router delete mode 100644 licenses-binary/LICENSE.ant-design-palettes delete mode 100644 licenses-binary/LICENSE.asm delete mode 100644 licenses-binary/LICENSE.base64 delete mode 100644 licenses-binary/LICENSE.bootstrap delete mode 100644 licenses-binary/LICENSE.cloudpickle delete mode 100644 licenses-binary/LICENSE.core-js delete mode 100644 licenses-binary/LICENSE.d3 delete mode 100644 licenses-binary/LICENSE.dagre delete mode 100644 licenses-binary/LICENSE.dagre-d3 delete mode 100644 licenses-binary/LICENSE.ev-emitter delete mode 100644 licenses-binary/LICENSE.font-awesome delete mode 100644 licenses-binary/LICENSE.google-auth-library-credentials delete mode 100644 licenses-binary/LICENSE.graphlib delete mode 100644 licenses-binary/LICENSE.grizzled-slf4j delete mode 100644 licenses-binary/LICENSE.imagesloaded delete mode 100644 licenses-binary/LICENSE.influx delete mode 100644 licenses-binary/LICENSE.janino delete mode 100644 licenses-binary/LICENSE.jline delete mode 100644 licenses-binary/LICENSE.jquery delete mode 100644 licenses-binary/LICENSE.jsr166y delete mode 100644 licenses-binary/LICENSE.jzlib delete mode 100644 licenses-binary/LICENSE.kryo delete mode 100644 licenses-binary/LICENSE.lodash delete mode 100644 licenses-binary/LICENSE.minlog delete mode 100644 licenses-binary/LICENSE.moment delete mode 100644 licenses-binary/LICENSE.moment-duration-format delete mode 100644 licenses-binary/LICENSE.monaco-editor delete mode 100644 licenses-binary/LICENSE.ng-zorro-antd delete mode 100644 licenses-binary/LICENSE.protobuf delete mode 100644 licenses-binary/LICENSE.py4j delete mode 100644 licenses-binary/LICENSE.pyrolite delete mode 100644 licenses-binary/LICENSE.qtip2 delete mode 100644 licenses-binary/LICENSE.rxjs delete mode 100644 licenses-binary/LICENSE.scala delete mode 100644 licenses-binary/LICENSE.scopt delete mode 100644 licenses-binary/LICENSE.slf4j delete mode 100644 licenses-binary/LICENSE.slf4j-api delete mode 100644 licenses-binary/LICENSE.split delete mode 100644 licenses-binary/LICENSE.tinycolor2 delete mode 100644 licenses-binary/LICENSE.tslib delete mode 100644 licenses-binary/LICENSE.webbit delete mode 100644 licenses-binary/LICENSE.zone diff --git a/NOTICE-binary b/NOTICE-binary deleted file mode 100644 index afea5236f2..0000000000 --- a/NOTICE-binary +++ /dev/null @@ -1,16732 +0,0 @@ -// ------------------------------------------------------------------ -// NOTICE file corresponding to the section 4d of The Apache License, -// Version 2.0, in this case for Apache Flink -// ------------------------------------------------------------------ - -Apache Flink -Copyright 2014-2019 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) - -- log4j:log4j:1.2.17 - -This project bundles the following dependencies under the MIT/X11 license. -See bundled license files for details. - -- org.slf4j:slf4j-log4j12:1.7.15 - -// ------------------------------------------------------------------ -// 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-2019 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 - -flink-connector-twitter -Copyright 2014-2019 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 - -Apache HttpCore -Copyright 2005-2017 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-azure-fs-hadoop -Copyright 2014-2019 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.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.google.guava:guava:11.0.2 -- com.microsoft.azure:azure-keyvault-core:0.8.0 -- com.microsoft.azure:azure-storage:5.4.0 -- commons-codec:commons-codec:1.10 -- commons-logging:commons-logging:1.1.3 -- org.apache.hadoop:hadoop-azure:3.1.0 -- org.apache.httpcomponents:httpclient:4.5.3 -- org.apache.httpcomponents:httpcore:4.4.6 -- org.eclipse.jetty:jetty-util:9.3.19.v20170502 -- org.eclipse.jetty:jetty-util-ajax:9.3.19.v20170502 - - -flink-hadoop-fs -Copyright 2014-2019 The Apache Software Foundation - -flink-fs-hadoop-shaded -Copyright 2014-2019 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). - -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 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-cep-scala -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-cep -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-dist -Copyright 2014-2019 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.data-artisans:frocksdbjni:5.17.2-artisans-1.0 -- 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.3.7 -- com.typesafe.akka:akka-actor_2.11:2.5.1 -- com.typesafe.akka:akka-camel_2.11:2.5.21 -- com.typesafe.akka:akka-protobuf_2.11:2.5.21 -- com.typesafe.akka:akka-slf4j_2.11:2.5.21 -- com.typesafe.akka:akka-stream_2.11:2.5.21 -- 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.18 -- 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.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.1.1 -- 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.2 - - -flink-core -Copyright 2014-2019 The Apache Software Foundation - -flink-annotations -Copyright 2014-2019 The Apache Software Foundation - -Apache Flink-shaded -Copyright 2006-2019 The Apache Software Foundation - -flink-shaded-asm7 -Copyright 2014-2018 The Apache Software Foundation - -- org.ow2.asm:asm:7.1 -- org.ow2.asm:asm-analysis:7.1 -- org.ow2.asm:asm-commons:7.1 -- org.ow2.asm:asm-tree:7.1 - -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()) - -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-2018 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (https://www.apache.org/). - -The files in the package org.apache.commons.compress.archivers.sevenz -were derived from the LZMA SDK, version 9.20 (C/ and CPP/7zip/), -which has been placed in the public domain: - -"LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html) - -flink-shaded-guava-18 -Copyright 2014-2018 The Apache Software Foundation - -- com.google.guava:guava:18.0 - -flink-java -Copyright 2014-2019 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-scala -Copyright 2014-2019 The Apache Software Foundation - -flink-runtime -Copyright 2014-2019 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.5.21 -- 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-2019 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 - -flink-queryable-state-client-java -Copyright 2014-2019 The Apache Software Foundation - -Apache Commons IO -Copyright 2002-2012 The Apache Software Foundation - -flink-shaded-netty -Copyright 2014-2018 The Apache Software Foundation - -This project includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -- io.netty:netty-all:4.1.39.Final - -flink-shaded-jackson -Copyright 2014-2018 The Apache Software Foundation - -- com.fasterxml.jackson.core:jackson-annotations:2.9.8 -- com.fasterxml.jackson.core:jackson-core:2.9.8 -- com.fasterxml.jackson.core:jackson-databind:2.9.8 -- com.fasterxml.jackson.dataformat:jackson-dataformat-csv:2.9.8 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.9.8 -- org.yaml:snakeyaml:1.18 - -# 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-runtime-web -Copyright 2014-2019 The Apache Software Foundation - --------------------------------------------------------------------------------------------------------------------- - -The old frontend bundles the following dependencies: - -- 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 - -- 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) - -The new frontend bundles the following dependencies: - -@angular/animations 7.2.10 : MIT License -@angular/cdk 7.3.6 : MIT License -@angular/common 7.2.10 : MIT License -@angular/compiler 7.2.10 : MIT License -@angular/core 7.2.10 : MIT License -@angular/forms 7.2.10 : MIT License -@angular/platform-browser 7.2.10 : MIT License -@angular/platform-browser-dynamic 7.2.10 : MIT License -@angular/router 7.2.10 : MIT License -@ant-design/icons-angular 2.0.2 : MIT License -@antv/adjust 0.1.1 : MIT License -@antv/attr 0.1.2 : MIT License -@antv/component 0.3.1 : MIT License -@antv/coord 0.1.0 : MIT License -@antv/g 3.3.6 : MIT License -@antv/g2 3.4.10 : MIT License -@antv/gl-matrix 2.7.1 : MIT License -@antv/scale 0.1.2 : MIT License -@antv/util 1.3.1 : ISC License -align-text 0.1.4 : MIT License -amdefine 1.0.1 : MIT License -ansi-regex 2.1.1 : MIT License -ant-design-palettes 1.1.3 : MIT License -balanced-match 1.0.0 : MIT License -brace-expansion 1.1.11 : MIT License -camelcase 1.2.1 : MIT License -center-align 0.1.3 : MIT License -Chalk 1.1.3 : MIT License -cliui 2.1.0 : ISC License -Commander.js 2.19.0 : MIT License -contour_plot 0.0.1 : MIT License -core-js v2.6.5 : MIT License -cpettitt/graphlib 2.1.7 : MIT License -d3-array 1.2.4 : BSD 3-clause "New" or "Revised" License -d3-axis 1.0.12 : BSD 3-clause "New" or "Revised" License -d3-brush 1.0.6 : BSD 3-clause "New" or "Revised" License -d3-chord 1.0.6 : BSD 3-clause "New" or "Revised" License -d3-collection 1.0.7 : BSD 3-clause "New" or "Revised" License -d3-color 1.2.3 : BSD 3-clause "New" or "Revised" License -d3-contour 1.3.2 : BSD 3-clause "New" or "Revised" License -d3-dispatch 1.0.5 : BSD 3-clause "New" or "Revised" License -d3-drag 1.2.3 : BSD 3-clause "New" or "Revised" License -d3-dsv 1.1.1 : BSD 3-clause "New" or "Revised" License -d3-ease 1.0.5 : BSD 3-clause "New" or "Revised" License -d3-fetch 1.1.2 : BSD 3-clause "New" or "Revised" License -d3-force 1.2.1 : BSD 3-clause "New" or "Revised" License -d3-format 1.3.2 : BSD 3-clause "New" or "Revised" License -d3-geo 1.11.3 : BSD 3-clause "New" or "Revised" License -d3-hierarchy 1.1.8 : BSD 3-clause "New" or "Revised" License -d3-interpolate v1.1.6 : BSD 3-clause "New" or "Revised" License -d3-path 1.0.7 : BSD 3-clause "New" or "Revised" License -d3-polygon v1.0.5 : BSD 3-clause "New" or "Revised" License -d3-quadtree 1.0.6 : BSD 3-clause "New" or "Revised" License -d3-random 1.1.2 : BSD 3-clause "New" or "Revised" License -d3-scale 2.2.2 : BSD 3-clause "New" or "Revised" License -d3-scale-chromatic 1.3.3 : BSD 3-clause "New" or "Revised" License -d3-selection v1.4.0 : BSD 3-clause "New" or "Revised" License -d3-shape v1.3.5 : BSD 3-clause "New" or "Revised" License -d3-time 1.0.11 : BSD 3-clause "New" or "Revised" License -d3-time-format 2.1.3 : BSD 3-clause "New" or "Revised" License -d3-timer v1.0.9 : BSD 3-clause "New" or "Revised" License -d3-transition v1.2.0 : BSD 3-clause "New" or "Revised" License -d3-voronoi v1.1.4 : (BSD 3-clause "New" or "Revised" License AND MIT License) -d3-zoom 1.7.3 : BSD 3-clause "New" or "Revised" License -D3.js 5.9.2 : BSD 3-clause "New" or "Revised" License -dagre 0.8.4 : MIT License -Decamelize 1.2.0 : MIT License -define-properties v1.1.3 : MIT License -defined 1.0.0 : MIT License -es-abstract 1.13.0 : MIT License -es-to-primitive 1.2.0 : MIT License -escape-string-regexp 1.0.5 : MIT License -fecha 2.3.3 : MIT License -fmin 0.0.2 : BSD 3-clause "New" or "Revised" License -fs.realpath 1.0.0 : ISC License -function-bind 1.1.1 : MIT License -has 1.0.3 : MIT License -has-ansi 2.0.0 : MIT License -has-symbols 1.0.0 : MIT License -iconv-lite v0.4.24 : MIT License -inflight 1.0.6 : ISC License -inherits 2.0.3 : ISC License -is-buffer 1.1.6 : MIT License -is-callable 1.1.4 : MIT License -is-date-object 1.0.1 : MIT License -is-regex 1.0.4 : MIT License -is-symbol 1.0.2 : MIT License -isaacs/once 1.4.0 : ISC License -json2module 0.0.3 : BSD 3-clause "New" or "Revised" License -kind-of 3.2.2 : MIT License -kossnocorp/date-fns 1.30.1 : MIT License -lazy-cache 1.0.4 : MIT License -Lo-Dash 4.17.11 : MIT License -longest 1.0.1 : MIT License -minimatch 3.0.4 : ISC License -minimist 1.2.0 : MIT License -monaco-editor 0.16.2 : MIT License -ng-zorro-antd 7.2.0 : MIT License -node-concat-map 0.0.1 : MIT License -node-deep-equal 1.0.1 : MIT License -node-glob v7.1.3 : ISC License -node-source-map-support 0.3.3 : MIT License -node-tape v4.10.1 : MIT License -object-inspect 1.6.0 : MIT License -object-keys 1.1.0 : MIT License -parse5 5.1.0 : MIT License -path-is-absolute 1.0.1 : MIT License -path-parse 1.0.6 : MIT License -Raynos/for-each 0.3.3 : MIT License -repeat-string 1.6.1 : MIT License -RESOLVE v1.10.0 : MIT License -resumer 0.0.0 : MIT License -right-align 0.1.3 : MIT License -rollup 0.25.8 : MIT License -rw 1.3.3 : BSD 3-clause "New" or "Revised" License -RxJS 6.3.3 : Apache License 2.0 - Copyright (c) 2015-2018 Google, Inc., Netflix, Inc., Microsoft Corp. and contributors - - No NOTICE file was provided by RxJS -safer-buffer 2.1.2 : MIT License -sindresorhus/ansi-styles 2.2.1 : MIT License -sindresorhus/supports-color 2.0.0 : MIT License -source-map 0.1.32 : BSD 3-clause "New" or "Revised" License -source-map 0.5.7 : BSD 3-clause "New" or "Revised" License -string.prototype.trim 1.1.2 : MIT License -Strip ANSI 3.0.1 : MIT License -through 2.3.8 : MIT License -TinyColor 1.4.1 : MIT License -tslib 1.9.3 : Apache License 2.0 - Copyright (c) Microsoft Corporation. All rights reserved. - - No NOTICE file was provided. -uglify-to-browserify 1.0.2 : MIT License -UglifyJS 2.8.29 : BSD 2-clause "Simplified" License -venn.js 0.2.20 : MIT License -window-size 0.1.0 : MIT License -wolfy87-eventemitter 5.1.0 : The Unlicense -wordwrap 0.0.2 : MIT License -wrappy 1.0.2 : ISC License -yargs 3.10.0 : MIT License -Zone.js v0.8.29 : MIT License - -Licenses: - -Apache License 2.0 -(RxJS 6.3.3, tslib 1.9.3) - -Apache License -Version 2.0, January 2004 -========================= - -http://www.apache.org/licenses/ - -TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - -1. Definitions. - -"License" shall mean the terms and conditions for use, reproduction, and -distribution as defined by Sections 1 through 9 of this document. - -"Licensor" shall mean the copyright owner or entity authorized by the copyright -owner that is granting the License. - -"Legal Entity" shall mean the union of the acting entity and all other entities -that control, are controlled by, or are under common control with that entity. -For the purposes of this definition, "control" means (i) the power, direct or -indirect, to cause the direction or management of such entity, whether by -contract or otherwise, or (ii) ownership of fifty percent (50%) or more of the -outstanding shares, or (iii) beneficial ownership of such entity. - -"You" (or "Your") shall mean an individual or Legal Entity exercising permissions -granted by this License. - -"Source" form shall mean the preferred form for making modifications, including -but not limited to software source code, documentation source, and configuration -files. - -"Object" form shall mean any form resulting from mechanical transformation or -translation of a Source form, including but not limited to compiled object code, -generated documentation, and conversions to other media types. - -"Work" shall mean the work of authorship, whether in Source or Object form, made -available under the License, as indicated by a copyright notice that is included -in or attached to the work (an example is provided in the Appendix below). - -"Derivative Works" shall mean any work, whether in Source or Object form, that is -based on (or derived from) the Work and for which the editorial revisions, -annotations, elaborations, or other modifications represent, as a whole, an -original work of authorship. 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 and Derivative Works thereof. - -"Contribution" shall mean any work of authorship, including the original version -of the Work and any modifications or additions to that Work or Derivative Works -thereof, that is intentionally submitted to Licensor for inclusion in the Work by -the copyright owner or by an individual or Legal Entity authorized to submit on -behalf of the copyright owner. For the purposes of this definition, "submitted" -means any form of electronic, verbal, or written communication sent to the -Licensor or its representatives, including but not limited to communication on -electronic mailing lists, source code control systems, and issue tracking systems -that are managed by, or on behalf of, the Licensor for the purpose of discussing -and improving the Work, but excluding communication that is conspicuously marked -or otherwise designated in writing by the copyright owner as "Not a -Contribution." - -"Contributor" shall mean Licensor and any individual or Legal Entity on behalf of -whom a Contribution has been received by Licensor and subsequently incorporated -within the Work. - -2. Grant of Copyright License. Subject to the terms and conditions of this -License, each Contributor hereby grants to You a perpetual, worldwide, -non-exclusive, no-charge, royalty-free, irrevocable copyright license to -reproduce, prepare Derivative Works of, publicly display, publicly perform, -sublicense, and distribute the Work and such Derivative Works in Source or Object -form. - -3. Grant of Patent License. Subject to the terms and conditions of this License, -each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, -no-charge, royalty-free, irrevocable (except as stated in this section) patent -license to make, have made, use, offer to sell, sell, import, and otherwise -transfer the Work, where such license applies only to those patent claims -licensable by such Contributor that are necessarily infringed by their -Contribution(s) alone or by combination of their Contribution(s) with the Work to -which such Contribution(s) was submitted. If You institute patent litigation -against any entity (including a cross-claim or counterclaim in a lawsuit) -alleging that the Work or a Contribution incorporated within the Work constitutes -direct or contributory patent infringement, then any patent licenses granted to -You under this License for that Work shall terminate as of the date such -litigation is filed. - -4. Redistribution. You may reproduce and distribute copies of the Work or -Derivative Works thereof in any medium, with or without modifications, and in -Source or Object form, provided that You meet the following conditions: - - a. You must give any other recipients of the Work or Derivative Works a copy of - this License; and - - b. You must cause any modified files to carry prominent notices stating that - You changed the files; and - - c. You must retain, in the Source form of any Derivative Works that You - distribute, all copyright, patent, trademark, and attribution notices from - the Source form of the Work, excluding those notices that do not pertain to - any part of the Derivative Works; and - - d. If the Work includes a "NOTICE" text file as part of its distribution, then - any Derivative Works that You distribute must include a readable copy of the - attribution notices contained within such NOTICE file, excluding those - notices that do not pertain to any part of the Derivative Works, in at least - one of the following places: within a NOTICE text file distributed as part of - the Derivative Works; within the Source form or documentation, if provided - along with the Derivative Works; or, within a display generated by the - Derivative Works, if and wherever such third-party notices normally appear. - The contents of the NOTICE file are for informational purposes only and do - not modify the License. You may add Your own attribution notices within - Derivative Works that You distribute, alongside or as an addendum to the - NOTICE text from the Work, provided that such additional attribution notices - cannot be construed as modifying the License. - -You may add Your own copyright statement to Your modifications and may provide -additional or different license terms and conditions for use, reproduction, or -distribution of Your modifications, or for any such Derivative Works as a whole, -provided Your use, reproduction, and distribution of the Work otherwise complies -with the conditions stated in this License. - -5. Submission of Contributions. Unless You explicitly state otherwise, any -Contribution intentionally submitted for inclusion in the Work by You to the -Licensor shall be under the terms and conditions of this License, without any -additional terms or conditions. Notwithstanding the above, nothing herein shall -supersede or modify the terms of any separate license agreement you may have -executed with Licensor regarding such Contributions. - -6. Trademarks. This License does not grant permission to use the trade names, -trademarks, service marks, or product names of the Licensor, except as required -for reasonable and customary use in describing the origin of the Work and -reproducing the content of the NOTICE file. - -7. Disclaimer of Warranty. Unless required by applicable law or agreed to in -writing, Licensor provides the Work (and each Contributor provides its -Contributions) 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. You are solely responsible for determining the -appropriateness of using or redistributing the Work and assume any risks -associated with Your exercise of permissions under this License. - -8. Limitation of Liability. In no event and under no legal theory, whether in -tort (including negligence), contract, or otherwise, unless required by -applicable law (such as deliberate and grossly negligent acts) or agreed to in -writing, shall any Contributor be liable to You for damages, including any -direct, indirect, special, incidental, or consequential damages of any character -arising as a result of this License or out of the use or inability to use the -Work (including but not limited to damages for loss of goodwill, work stoppage, -computer failure or malfunction, or any and all other commercial damages or -losses), even if such Contributor has been advised of the possibility of such -damages. - -9. Accepting Warranty or Additional Liability. While redistributing the Work or -Derivative Works thereof, You may choose to offer, and charge a fee for, -acceptance of support, warranty, indemnity, or other liability obligations and/or -rights consistent with this License. However, in accepting such obligations, You -may act only on Your own behalf and on Your sole responsibility, not on behalf of -any other Contributor, and only if You agree to indemnify, defend, and hold each -Contributor harmless for any liability incurred by, or claims asserted against, -such Contributor by reason of your accepting any such warranty or additional -liability. - -END OF TERMS AND CONDITIONS - -APPENDIX: How to apply the Apache License to your work - -To apply the Apache License to your work, attach the following boilerplate -notice, with the fields enclosed by brackets "[]" replaced with your own -identifying information. (Don't include the brackets!) The text should be -enclosed in the appropriate comment syntax for the file format. We also recommend -that a file or class name and description of purpose be included on the same -"printed page" as the copyright notice for easier identification within -third-party archives. - - Copyright [yyyy] [name of copyright owner] 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. - ---- - -BSD 2-clause "Simplified" License -(UglifyJS 2.8.29) - -UglifyJS is released under the BSD license: - -Copyright 2012-2013 (c) Mihai Bazon - -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 HOLDER “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 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 - -BSD 3-clause "New" or "Revised" License -(d3-array 1.2.4, d3-axis 1.0.12, d3-brush 1.0.6, d3-chord 1.0.6, d3-collection 1.0.7, d3-color 1.2.3, d3-contour 1.3.2, d3-dispatch 1.0.5, d3-drag 1.2.3, d3-dsv 1.1.1, d3-ease 1.0.5, d3-fetch 1.1.2, d3-force 1.2.1, d3-format 1.3.2, d3-geo 1.11.3, d3-hierarchy 1.1.8, d3-interpolate v1.1.6, d3-path 1.0.7, d3-polygon v1.0.5, d3-quadtree 1.0.6, d3-random 1.1.2, d3-scale 2.2.2, d3-scale-chromatic 1.3.3, d3-selection v1.4.0, d3-shape v1.3.5, d3-time 1.0.11, d3-time-format 2.1.3, d3-timer v1.0.9, d3-transition v1.2.0, d3-voronoi v1.1.4, d3-zoom 1.7.3, D3.js 5.9.2, fmin 0.0.2, json2module 0.0.3, source-map 0.1.32) - -Copyright (c) , -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 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. - -BSD 3-clause "New" or "Revised" License -(source-map 0.5.7) - -License: BSD-3-clause - -Files: debian/* -Copyright: 2014 Leo Iannacone -License: BSD-3-clause - -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 University 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 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 - -BSD 3-clause "New" or "Revised" License -(rw 1.3.3) - -Upstream-Contact: https://github.com/mbostock/rw/issues -Source: https://github.com/mbostock/rw - -Files: * -Copyright: 2014-2016 Mike Bostock (http://bost.ocks.org/mike) -License: BSD-3-Clause - -Files: debian/* -Copyright: 2017 Pirate Praveen -License: BSD-3-Clause - -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 University 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 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 - -ISC License -(cliui 2.1.0) - -Copyright (c) 2015, Contributors - -Permission to use, copy, modify, and/or distribute this software -for any purpose with or without fee is hereby granted, provided -that the above copyright notice and 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 - -ISC License -(@antv/util 1.3.1) - -ISC License (ISCL) -================== - -Copyright (c) 4-digit year, Company or Person's Name - -Permission to use, copy, modify, and/or distribute this software for any purpose -with or without fee is hereby granted, provided that the above copyright notice -and 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. - -ISC License -(inflight 1.0.6) - -The ISC License - -Copyright (c) Isaac Z. Schlueter - -Permission to use, copy, modify, and/or distribute this software for any -purpose with or without fee is hereby granted, provided that the above -copyright notice and 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 - -ISC License -(inherits 2.0.3) - -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 - -ISC License -(fs.realpath 1.0.0, isaacs/once 1.4.0, minimatch 3.0.4, node-glob v7.1.3) - -Copyright (c) Isaac Z. Schlueter and Contributors - -ISC License -(wrappy 1.0.2) - -Upstream-Contact: https://github.com/npm/wrappy/issues -Source: https://github.com/npm/wrappy - -Files: * -Copyright: 2015 Isaac Z. Schlueter (http://blog.izs.me/) -License: ISC - -Files: debian/* -Copyright: 2015 Thorsten Alteholz -License: ISC - -License: ISC - -Permission to use, copy, modify, and/or distribute this software for any - purpose with or without fee is hereby granted, provided that the above - copyright notice and 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 - -MIT License -(Commander.js 2.19.0) - -(The MIT License) - -Copyright (c) 2011 TJ Holowaychuk - -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 - -MIT License -(Chalk 1.1.3) - -2016, Mathias Behrle -License: Expat - -License: Expat - -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 - -MIT License -(Lo-Dash 4.17.11) - -==== - -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 - -MIT License -(d3-voronoi v1.1.4) - -Copyright (C) 2010-2013 Raymond Hill -https://github.com/gorhill/Javascript-Voronoi - -Licensed under The MIT License -http://en.wikipedia.org/wiki/MIT_License - -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 - -MIT License -(iconv-lite v0.4.24) - -Copyright (c) 2011 Alexander Shtuchkin - -MIT License -(venn.js 0.2.20) - -Copyright (C) 2013 Ben Frederickson - -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 - -MIT License -(uglify-to-browserify 1.0.2) - -Copyright (c) 2013 Forbes Lindesay - -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 - -MIT License -(balanced-match 1.0.0) - -Copyright (c) 2013 Julian Gruber <julian@juliangruber.com> - -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 - -MIT License -(has 1.0.3) - -Copyright (c) 2013 Thiago de Arruda - -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 - -MIT License -(parse5 5.1.0) - -Copyright (c) 2013-2018 Ivan Nikulin (ifaaan@gmail.com, https://github.com/inikulin) - -MIT License -(window-size 0.1.0) - -Copyright (c) 2014 Jon Schlinkert - -MIT License -(TinyColor 1.4.1) - -Copyright (c), Brian Grinstead, http://briangrinstead.com - -MIT License -(wordwrap 0.0.2) - -Files: * -Copyright: © 2010, James Halliday -License: MIT - -Files: debian/* -Copyright: © 2011, David Paleino -License: 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 - -MIT License -(function-bind 1.1.1) - -MIT License -(brace-expansion 1.1.11) - -MIT License - -Copyright (c) 2013 Julian Gruber - -MIT License -(has-symbols 1.0.0) - -Copyright (c) 2016 Jordan Harband - -MIT License -(safer-buffer 2.1.2) - -Copyright (c) 2018 Nikita Skovoroda - -MIT License -(amdefine 1.0.1) - -MIT License ------------ - -Copyright (c) 2011-2016, The Dojo Foundation - -MIT License -(yargs 3.10.0) - -MIT License -(through 2.3.8) - -The MIT License - -Copyright (c) 2011 Dominic Tarr - -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 - -MIT License -(@angular/animations 7.2.10, @angular/cdk 7.3.6, @angular/common 7.2.10, @angular/compiler 7.2.10, @angular/core 7.2.10, @angular/forms 7.2.10, @angular/platform-browser 7.2.10, @angular/platform-browser-dynamic 7.2.10, @angular/router 7.2.10, @ant-design/icons-angular 2.0.2, @antv/adjust 0.1.1, @antv/attr 0.1.2, @antv/component 0.3.1, @antv/coord 0.1.0, @antv/g 3.3.6, @antv/g2 3.4.10, @antv/gl-matrix 2.7.1, @antv/scale 0.1.2, ant-design-palettes 1.1.3, contour_plot 0.0.1, core-js v2.6.5, cpettitt/graphlib 2.1.7, dagre 0.8.4, es-abstract 1.13.0, is-callable 1.1.4, kossnocorp/date-fns 1.30.1, monaco-editor 0.16.2, ng-zorro-antd 7.2.0, node-tape v4.10.1, object-keys 1.1.0, RESOLVE v1.10.0, Zone.js v0.8.29) - -The MIT License -=============== - -Copyright (c) - -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. - -MIT License -(Raynos/for-each 0.3.3) - -The MIT License (MIT) - -Copyright (c) 2012 Raynos. - -MIT License -(node-source-map-support 0.3.3) - -Copyright (c) 2014 Evan Wallace - -MIT License -(is-regex 1.0.4) - -Copyright (c) 2014 Jordan Harband - -MIT License -(longest 1.0.1) - -Copyright (c) 2014-2015, Jon Schlinkert. - -MIT License -(repeat-string 1.6.1) - -Copyright (c) 2014-2016, Jon Schlinkert. - -MIT License -(kind-of 3.2.2) - -Copyright (c) 2014-2017, Jon Schlinkert - -MIT License -(rollup 0.25.8) - -Copyright (c) 2015 [these people](https://github.com/rollup/rollup/graphs/contributors) - -MIT License -(path-parse 1.0.6) - -Copyright (c) 2015 Javier Blanco - -MIT License -(define-properties v1.1.3) - -Copyright (C) 2015 Jordan Harband - -MIT License -(es-to-primitive 1.2.0, is-date-object 1.0.1, is-symbol 1.0.2, string.prototype.trim 1.1.2) - -Copyright (c) 2015 Jordan Harband - -MIT License -(fecha 2.3.3) - -Copyright (c) 2015 Taylor Hakes - -MIT License -(align-text 0.1.4, center-align 0.1.3, right-align 0.1.3) - -Copyright (c) 2015, Jon Schlinkert. - -MIT License -(lazy-cache 1.0.4) - -Copyright (c) 2015-2016, Jon Schlinkert. - -MIT License -(is-buffer 1.1.6) - -Copyright (c) Feross Aboukhadijeh - -MIT License -(ansi-regex 2.1.1, camelcase 1.2.1, Decamelize 1.2.0, escape-string-regexp 1.0.5, has-ansi 2.0.0, path-is-absolute 1.0.1, sindresorhus/ansi-styles 2.2.1, sindresorhus/supports-color 2.0.0, Strip ANSI 3.0.1) - -Copyright (c) Sindre Sorhus (sindresorhus.com) - -MIT License -(defined 1.0.0, minimist 1.2.0, node-concat-map 0.0.1, node-deep-equal 1.0.1, object-inspect 1.6.0, resumer 0.0.0) - -This software is released under the MIT license: - -The Unlicense -(wolfy87-eventemitter 5.1.0) - -The Unlicense -============= - -This is free and unencumbered software released into the public domain. - -Anyone is free to copy, modify, publish, use, compile, sell, or distribute this -software, either in source code form or as a compiled binary, for any purpose, -commercial or non-commercial, and by any means. - -In jurisdictions that recognize copyright laws, the author or authors of this -software dedicate any and all copyright interest in the software to the public -domain. We make this dedication for the benefit of the public at large and to the -detriment of our heirs and successors. We intend this dedication to be an overt -act of relinquishment in perpetuity of all present and future rights to this -software under copyright law. - -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 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 more information, please refer to http://unlicense.org/ - -flink-optimizer -Copyright 2014-2019 The Apache Software Foundation - -flink-clients -Copyright 2014-2019 The Apache Software Foundation - -flink-streaming-java -Copyright 2014-2019 The Apache Software Foundation - -flink-streaming-scala -Copyright 2014-2019 The Apache Software Foundation - -flink-metrics-core -Copyright 2014-2019 The Apache Software Foundation - -flink-metrics-jmx -Copyright 2014-2019 The Apache Software Foundation - -flink-mesos -Copyright 2014-2019 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-container -Copyright 2014-2019 The Apache Software Foundation - -flink-statebackend-rocksdb -Copyright 2014-2019 The Apache Software Foundation - -flink-yarn -Copyright 2014-2019 The Apache Software Foundation - -flink-hadoop-fs -Copyright 2014-2019 The Apache Software Foundation - -flink-mapr-fs -Copyright 2014-2019 The Apache Software Foundation - -flink-scala-shell -Copyright 2014-2019 The Apache Software Foundation - -force-shading -Copyright 2019 The Apache Software Foundation - -Objenesis -Copyright 2006-2013 Joe Walnes, Henri Tremblay, Leonardo Mesquita - -// ------------------------------------------------------------------ -// 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-gelly-examples -Copyright 2014-2019 The Apache Software Foundation - - -force-shading -Copyright 2019 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - - -// ------------------------------------------------------------------ -// 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-gelly -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-metrics-datadog -Copyright 2014-2019 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 - -// ------------------------------------------------------------------ -// 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-metrics-graphite -Copyright 2014-2019 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 - - -flink-metrics-dropwizard -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-metrics-influxdb -Copyright 2014-2019 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.moshi:moshi:1.5.0 -- com.squareup.okhttp3:logging-interceptor:3.11.0 -- com.squareup.okhttp3:okhttp:3.11.0 -- com.squareup.okio:okio:1.14.0 -- com.squareup.retrofit2:converter-moshi:2.4.0 -- com.squareup.retrofit2:retrofit:2.4.0 - -This project bundles the following dependencies under the MIT license. (https://opensource.org/licenses/MIT) - -- org.influxdb:influxdb-java:2.14 - -// ------------------------------------------------------------------ -// 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-metrics-prometheus -Copyright 2014-2019 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 - -// ------------------------------------------------------------------ -// 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-metrics-slf4j -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-metrics-statsd -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-oss-fs-hadoop -Copyright 2014-2019 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.aliyun.oss:aliyun-sdk-oss:3.4.1 -- com.aliyun:aliyun-java-sdk-core:3.4.0 -- com.aliyun:aliyun-java-sdk-ecs:4.2.0 -- com.aliyun:aliyun-java-sdk-ram:3.0.0 -- com.aliyun:aliyun-java-sdk-sts:3.0.0 -- 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 -- com.github.stephenc.jcip:jcip-annotations:1.0-1 -- com.google.code.gson:gson:2.2.4 -- com.google.guava:guava:11.0.2 -- com.nimbusds:nimbus-jose-jwt:4.41.1 -- 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-io:commons-io:2.4 -- commons-lang:commons-lang:3.3.2 -- commons-logging:commons-logging:1.1.3 -- commons-net:commons-net:3.6 -- net.minidev:accessors-smart:1.2 -- net.minidev:json-smart:2.3 -- org.apache.avro:avro:1.8.2 -- org.apache.commons:commons-compress:1.18 -- org.apache.commons:commons-configuration2:2.1.1 -- org.apache.commons:commons-lang3:3.3.2 -- org.apache.commons:commons-math3:3.5 -- org.apache.curator:curator-client:2.12.0 -- org.apache.curator:curator-framework:2.12.0 -- org.apache.curator:curator-recipes:2.12.0 -- org.apache.hadoop:hadoop-aliyun:3.1.0 -- 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.httpcomponents:httpclient:4.5.3 -- org.apache.httpcomponents:httpcore:4.4.6 -- org.apache.kerby:kerb-admin:1.0.1 -- org.apache.kerby:kerb-client:1.0.1 -- org.apache.kerby:kerb-common:1.0.1 -- org.apache.kerby:kerb-core:1.0.1 -- org.apache.kerby:kerb-crypto:1.0.1 -- org.apache.kerby:kerb-identity:1.0.1 -- org.apache.kerby:kerb-server:1.0.1 -- org.apache.kerby:kerb-simplekdc:1.0.1 -- org.apache.kerby:kerb-util:1.0.1 -- org.apache.kerby:kerby-asn1:1.0.1 -- org.apache.kerby:kerby-config:1.0.1 -- org.apache.kerby:kerby-pkix:1.0.1 -- org.apache.kerby:kerby-util:1.0.1 -- org.apache.kerby:kerby-xdr:1.0.1 -- org.apache.kerby:token-provider:1.0.1 -- org.apache.zookeeper:zookeeper:3.4.10 -- org.codehaus.jackson:jackson-core-asl:1.9.2 -- org.codehaus.jackson:jackson-jaxrs:1.9.2 -- org.codehaus.jackson:jackson-mapper-asl:1.9.2 -- org.codehaus.jackson:jackson-xc:1.9.2 -- org.codehaus.jettison:jettison:1.1 -- org.eclipse.jetty:jetty-http:9.3.19.v20170502 -- org.eclipse.jetty:jetty-io:9.3.19.v20170502 -- org.eclipse.jetty:jetty-security:9.3.19.v20170502 -- org.eclipse.jetty:jetty-server:9.3.19.v20170502 -- org.eclipse.jetty:jetty-servlet:9.3.19.v20170502 -- org.eclipse.jetty:jetty-util:9.3.19.v20170502 -- org.eclipse.jetty:jetty-webapp:9.3.19.v20170502 -- org.eclipse.jetty:jetty-xml:9.3.19.v20170502 -- 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.google.protobuf:protobuf-java:2.5.0 -- com.jcraft:jsch:0.1.54 -- com.thoughtworks.paranamer:paranamer:2.7 -- org.codehaus.woodstox:stax2-api:3.1.4 -- org.ow2.asm:asm:5.0.4 - -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 -- javax.ws.rs:jsr311-api:1.1.1 -- javax.xml.stream:stax-api:1.0-2 -- stax:stax-api:1.0.1 - -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-core:1.19 -- com.sun.jersey:jersey-json:1.9 -- com.sun.jersey:jersey-server:1.19 -- com.sun.jersey:jersey-servlet:1.19 -- com.sun.xml.bind:jaxb-impl:2.2.3-1 -- javax.servlet.jsp:jsp-api:2.1 -- javax.servlet:javax.servlet-api:3.1.0 -- javax.xml.bind:jaxb-api:2.2.2 - -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 the JDOM license. -See bundled license files for details. - -- org.jdom:jdom:1.1 - -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. - - -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()) - -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-2018 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (https://www.apache.org/). - -The files in the package org.apache.commons.compress.archivers.sevenz -were derived from the LZMA SDK, version 9.20 (C/ and CPP/7zip/), -which has been placed in the public domain: - -"LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html) - -flink-hadoop-fs -Copyright 2014-2019 The Apache Software Foundation - -flink-fs-hadoop-shaded -Copyright 2014-2019 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 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 - -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 - -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). - -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 Net -Copyright 2001-2017 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 Avro -Copyright 2009-2017 The Apache Software Foundation - -Curator Framework -Copyright 2011-2017 The Apache Software Foundation - -Curator Client -Copyright 2011-2017 The Apache Software Foundation - -Curator Recipes -Copyright 2011-2017 The Apache Software Foundation - -Kerb Simple Kdc -Copyright 2014-2017 The Apache Software Foundation - -Kerby-kerb Client -Copyright 2014-2017 The Apache Software Foundation - -Kerby Config -Copyright 2014-2017 The Apache Software Foundation - -Kerby-kerb core -Copyright 2014-2017 The Apache Software Foundation - -Kerby PKIX Project -Copyright 2014-2017 The Apache Software Foundation - -Kerby ASN1 Project -Copyright 2014-2017 The Apache Software Foundation - -Kerby Util -Copyright 2014-2017 The Apache Software Foundation - -Kerby-kerb Common -Copyright 2014-2017 The Apache Software Foundation - -Kerby-kerb Crypto -Copyright 2014-2017 The Apache Software Foundation - -Kerby-kerb Util -Copyright 2014-2017 The Apache Software Foundation - -Token provider -Copyright 2014-2017 The Apache Software Foundation - -Kerby-kerb Admin -Copyright 2014-2017 The Apache Software Foundation - -Kerby-kerb Server -Copyright 2014-2017 The Apache Software Foundation - -Kerby-kerb Identity -Copyright 2014-2017 The Apache Software Foundation - -Kerby XDR Project -Copyright 2014-2017 The Apache Software Foundation - -Apache HttpClient -Copyright 1999-2017 The Apache Software Foundation - -Apache HttpCore -Copyright 2005-2017 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-python -Copyright 2014-2019 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.fasterxml.jackson.core:jackson-annotations:2.9.9 -- com.fasterxml.jackson.core:jackson-core:2.9.9 -- com.fasterxml.jackson.core:jackson-databind:2.9.9 -- com.google.api.grpc:proto-google-common-protos:1.12.0 -- com.google.code.gson:gson:2.7 -- com.google.guava:guava:26.0-jre -- io.grpc:grpc-auth:1.21.0 -- io.grpc:grpc-core:1.21.0 -- io.grpc:grpc-context:1.21.0 -- io.grpc:grpc-netty:1.21.0 -- io.grpc:grpc-protobuf:1.21.0 -- io.grpc:grpc-stub:1.21.0 -- io.grpc:grpc-testing:1.21.0 -- io.netty:netty-buffer:4.1.34.Final -- io.netty:netty-codec:4.1.34.Final -- io.netty:netty-codec-http:4.1.34.Final -- io.netty:netty-codec-http2:4.1.34.Final -- io.netty:netty-codec-socks:4.1.34.Final -- io.netty:netty-common:4.1.34.Final -- io.netty:netty-handler:4.1.34.Final -- io.netty:netty-handler-proxy:4.1.34.Final -- io.netty:netty-resolver:4.1.34.Final -- io.netty:netty-transport:4.1.34.Final -- io.netty:netty-transport-native-epoll:4.1.34.Final -- io.netty:netty-transport-native-unix-common:4.1.34.Final -- io.netty:netty-tcnative-boringssl-static:2.0.22.Final -- io.opencensus:opencensus-api:0.21.0 -- io.opencensus:opencensus-contrib-grpc-metrics:0.21.0 -- joda-time:joda-time:2.5 -- org.apache.beam:beam-model-fn-execution:2.15.0 -- org.apache.beam:beam-model-job-management:2.15.0 -- org.apache.beam:beam-model-pipeline:2.15.0 -- org.apache.beam:beam-runners-core-construction-java:2.15.0 -- org.apache.beam:beam-runners-java-fn-execution:2.15.0 -- org.apache.beam:beam-sdks-java-core:2.15.0 -- org.apache.beam:beam-sdks-java-fn-execution:2.15.0 -- org.apache.beam:beam-vendor-sdks-java-extensions-protobuf:2.15.0 -- org.apache.beam:beam-vendor-guava-26_0-jre:0.1 -- org.apache.beam:beam-vendor-grpc-1_21_0:0.1 - -This project bundles the following dependencies under the BSD license. -See bundled license files for details - -- net.sf.py4j:py4j:0.10.8.1 -- com.google.protobuf:protobuf-java:3.7.1 -- com.google.protobuf:protobuf-java-util:3.7.1 -- com.google.auth:google-auth-library-credentials:0.13.0 -- cloudpickle:1.2.2 - -This project bundles the following dependencies under the MIT license. (https://opensource.org/licenses/MIT) -See bundled license files for details. - -- net.razorvine:pyrolite:4.13 - -Apache Beam -Copyright 2016-2018 The Apache Software Foundation - -Based on source code originally developed by -Google (http://www.google.com/). - -This product includes software developed at -Google (http://www.google.com/). - -# 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). - -============================================================================= -= 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/). - -Apache Commons Compress -Copyright 2002-2018 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (https://www.apache.org/). - -The files in the package org.apache.commons.compress.archivers.sevenz -were derived from the LZMA SDK, version 9.20 (C/ and CPP/7zip/), -which has been placed in the public domain: - -"LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html) - -// ------------------------------------------------------------------ -// 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-queryable-state-runtime -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-s3-fs-hadoop -Copyright 2014-2019 The Apache Software Foundation - -flink-s3-fs-base -Copyright 2014-2019 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-2019 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). - -flink-hadoop-fs -Copyright 2014-2019 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/). - -// ------------------------------------------------------------------ -// 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-s3-fs-presto -Copyright 2014-2019 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.187 -- com.facebook.presto.hadoop:hadoop-apache2:2.7.3-1 -- com.google.guava:guava:21.0 -- io.airlift:configuration:0.153 -- io.airlift:log:0.153 -- io.airlift:stats:0.153 -- 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-2019 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-2019 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). - -flink-hadoop-fs -Copyright 2014-2019 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/). - -// ------------------------------------------------------------------ -// NOTICE file corresponding to the section 4d of The Apache License, -// Version 2.0, in this case for Apache Flink-shaded -// ------------------------------------------------------------------ - -Apache Flink-shaded -Copyright 2006-2019 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -flink-shaded-netty-openssl-static -Copyright 2014-2019 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) - -- io.netty:netty-tcnative:2.0.25.Final - -// ------------------------------------------------------------------ -// 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-sql-client -Copyright 2014-2019 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 - -// ------------------------------------------------------------------ -// 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-state-processor-api -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-swift-fs-hadoop -Copyright 2014-2019 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.18 -- 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-2018 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (https://www.apache.org/). - -The files in the package org.apache.commons.compress.archivers.sevenz -were derived from the LZMA SDK, version 9.20 (C/ and CPP/7zip/), -which has been placed in the public domain: - -"LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html) - -flink-hadoop-fs -Copyright 2014-2019 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 - -// ------------------------------------------------------------------ -// 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-table-uber-blink -Copyright 2014-2019 The Apache Software Foundation - -flink-table-common -Copyright 2014-2019 The Apache Software Foundation - -flink-table-api-java -Copyright 2014-2019 The Apache Software Foundation - -flink-table-api-scala -Copyright 2014-2019 The Apache Software Foundation - -flink-table-api-java-bridge -Copyright 2014-2019 The Apache Software Foundation - -flink-table-api-scala-bridge -Copyright 2014-2019 The Apache Software Foundation - -flink-table-planner-blink -Copyright 2014-2019 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:19.0 -- com.fasterxml.jackson.core:jackson-annotations:2.9.6 -- com.fasterxml.jackson.core:jackson-core:2.9.6 -- com.fasterxml.jackson.core:jackson-databind:2.9.6 -- com.jayway.jsonpath:json-path:2.4.0 -- joda-time:joda-time:2.5 -- org.apache.calcite:calcite-core:1.21.0 -- org.apache.calcite:calcite-linq4j:1.21.0 -- org.apache.calcite.avatica:avatica-core:1.15.0 -- commons-codec:commons-codec:1.10 - -This project bundles the following dependencies under the BSD license. -See bundled license files for details - -- org.codehaus.janino:janino:3.0.9 -- org.codehaus.janino:commons-compiler:3.0.9 - -flink-sql-parser -Copyright 2014-2019 The Apache Software Foundation - -Apache Calcite Avatica -Copyright 2012-2019 The Apache Software Foundation - -Calcite Core -Copyright 2012-2019 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. - -Calcite Linq4j -Copyright 2012-2019 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). - -flink-table-runtime-blink -Copyright 2014-2019 The Apache Software Foundation - -- org.lz4:lz4-java:1.5.0 -- org.apache.calcite.avatica:avatica-core:1.13.0 - -flink-cep -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-table-uber -Copyright 2014-2019 The Apache Software Foundation - -flink-table-common -Copyright 2014-2019 The Apache Software Foundation - -flink-table-api-java -Copyright 2014-2019 The Apache Software Foundation - -flink-table-api-scala -Copyright 2014-2019 The Apache Software Foundation - -flink-table-api-java-bridge -Copyright 2014-2019 The Apache Software Foundation - -flink-table-api-scala-bridge -Copyright 2014-2019 The Apache Software Foundation - -flink-table-planner -Copyright 2014-2019 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:19.0 -- com.fasterxml.jackson.core:jackson-annotations:2.9.6 -- com.fasterxml.jackson.core:jackson-core:2.9.6 -- com.fasterxml.jackson.core:jackson-databind:2.9.6 -- com.jayway.jsonpath:json-path:2.4.0 -- joda-time:joda-time:2.5 -- org.apache.calcite:calcite-core:1.21.0 -- org.apache.calcite:calcite-linq4j:1.21.0 -- org.apache.calcite.avatica:avatica-core:1.15.0 -- commons-codec:commons-codec:1.10 - -This project bundles the following dependencies under the BSD license. -See bundled license files for details - -- org.codehaus.janino:janino:3.0.9 -- org.codehaus.janino:commons-compiler:3.0.9 - -flink-sql-parser -Copyright 2014-2019 The Apache Software Foundation - -Calcite Core -Copyright 2012-2019 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 Calcite Avatica -Copyright 2012-2019 The Apache Software Foundation - -Calcite Linq4j -Copyright 2012-2019 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). - -============================================================================= -= 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-cep -Copyright 2014-2019 The Apache Software Foundation - -Apache log4j -Copyright 2007 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/flink-dist/src/main/assemblies/bin.xml b/flink-dist/src/main/assemblies/bin.xml index 8c549fbce8..9c1d29ba8a 100644 --- a/flink-dist/src/main/assemblies/bin.xml +++ b/flink-dist/src/main/assemblies/bin.xml @@ -79,21 +79,9 @@ under the License. 0644 - - - ../NOTICE-binary - - NOTICE - 0644 - - - ../licenses-binary - licenses - 0644 - src/main/flink-bin/bin diff --git a/licenses-binary/LICENSE-hdrhistogram b/licenses-binary/LICENSE-hdrhistogram deleted file mode 100644 index 09c38ea516..0000000000 --- a/licenses-binary/LICENSE-hdrhistogram +++ /dev/null @@ -1,125 +0,0 @@ -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 deleted file mode 100644 index 21645bec9d..0000000000 --- a/licenses-binary/LICENSE-protobuf +++ /dev/null @@ -1,36 +0,0 @@ -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 deleted file mode 100644 index b620ae68fe..0000000000 --- a/licenses-binary/LICENSE-re2j +++ /dev/null @@ -1,32 +0,0 @@ -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 deleted file mode 100644 index 0ed6361699..0000000000 --- a/licenses-binary/LICENSE-stax2api +++ /dev/null @@ -1,22 +0,0 @@ -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 deleted file mode 100644 index a1256bdf6f..0000000000 --- a/licenses-binary/LICENSE-xmlenc +++ /dev/null @@ -1,28 +0,0 @@ -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 deleted file mode 100644 index 828d3633f7..0000000000 --- a/licenses-binary/LICENSE.@angular +++ /dev/null @@ -1,21 +0,0 @@ -The MIT License - -Copyright (c) 2010-2019 Google LLC. http://angular.io/license - -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 b/licenses-binary/LICENSE.angular deleted file mode 100644 index 91f0644936..0000000000 --- a/licenses-binary/LICENSE.angular +++ /dev/null @@ -1,22 +0,0 @@ -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 deleted file mode 100644 index 1787fc65be..0000000000 --- a/licenses-binary/LICENSE.angular-drag-and-drop-list +++ /dev/null @@ -1,22 +0,0 @@ -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 deleted file mode 100644 index fe9db79b91..0000000000 --- a/licenses-binary/LICENSE.angular-moment +++ /dev/null @@ -1,21 +0,0 @@ -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 deleted file mode 100644 index 6413b092d7..0000000000 --- a/licenses-binary/LICENSE.angular-ui-router +++ /dev/null @@ -1,21 +0,0 @@ -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.ant-design-palettes b/licenses-binary/LICENSE.ant-design-palettes deleted file mode 100644 index dfe5c2183e..0000000000 --- a/licenses-binary/LICENSE.ant-design-palettes +++ /dev/null @@ -1,22 +0,0 @@ -MIT LICENSE - -Copyright (c) 2018-present Ant UED, https://xtech.antfin.com/ - -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.asm b/licenses-binary/LICENSE.asm deleted file mode 100644 index 62ffbccb68..0000000000 --- a/licenses-binary/LICENSE.asm +++ /dev/null @@ -1,31 +0,0 @@ -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 deleted file mode 100644 index 31ebc84053..0000000000 --- a/licenses-binary/LICENSE.base64 +++ /dev/null @@ -1,26 +0,0 @@ -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 deleted file mode 100644 index 6ca0ceecc5..0000000000 --- a/licenses-binary/LICENSE.bootstrap +++ /dev/null @@ -1,21 +0,0 @@ -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.cloudpickle b/licenses-binary/LICENSE.cloudpickle deleted file mode 100644 index d112c4806a..0000000000 --- a/licenses-binary/LICENSE.cloudpickle +++ /dev/null @@ -1,32 +0,0 @@ -This module was extracted from the `cloud` package, developed by -PiCloud, Inc. - -Copyright (c) 2015, Cloudpickle contributors. -Copyright (c) 2012, Regents of the University of California. -Copyright (c) 2009 PiCloud, Inc. http://www.picloud.com. -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 of California, Berkeley 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.core-js b/licenses-binary/LICENSE.core-js deleted file mode 100644 index 8cc76f6c18..0000000000 --- a/licenses-binary/LICENSE.core-js +++ /dev/null @@ -1,21 +0,0 @@ -The MIT License - -Copyright (c) 2019 Google LLC. - -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.d3 b/licenses-binary/LICENSE.d3 deleted file mode 100644 index 1d9d875edb..0000000000 --- a/licenses-binary/LICENSE.d3 +++ /dev/null @@ -1,27 +0,0 @@ -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 deleted file mode 100644 index 7d7dd94248..0000000000 --- a/licenses-binary/LICENSE.dagre +++ /dev/null @@ -1,19 +0,0 @@ -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. diff --git a/licenses-binary/LICENSE.dagre-d3 b/licenses-binary/LICENSE.dagre-d3 deleted file mode 100644 index 1d64ed68ce..0000000000 --- a/licenses-binary/LICENSE.dagre-d3 +++ /dev/null @@ -1,19 +0,0 @@ -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 deleted file mode 100644 index 3e30b1aeff..0000000000 --- a/licenses-binary/LICENSE.ev-emitter +++ /dev/null @@ -1,7 +0,0 @@ -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 deleted file mode 100644 index b9fb2c6e6e..0000000000 --- a/licenses-binary/LICENSE.font-awesome +++ /dev/null @@ -1,97 +0,0 @@ -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.google-auth-library-credentials b/licenses-binary/LICENSE.google-auth-library-credentials deleted file mode 100644 index 12edf23c67..0000000000 --- a/licenses-binary/LICENSE.google-auth-library-credentials +++ /dev/null @@ -1,28 +0,0 @@ -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. diff --git a/licenses-binary/LICENSE.graphlib b/licenses-binary/LICENSE.graphlib deleted file mode 100644 index e3c8f95557..0000000000 --- a/licenses-binary/LICENSE.graphlib +++ /dev/null @@ -1,19 +0,0 @@ -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 deleted file mode 100644 index 6601c0131e..0000000000 --- a/licenses-binary/LICENSE.grizzled-slf4j +++ /dev/null @@ -1,11 +0,0 @@ -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 deleted file mode 100644 index 3e30b1aeff..0000000000 --- a/licenses-binary/LICENSE.imagesloaded +++ /dev/null @@ -1,7 +0,0 @@ -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.influx b/licenses-binary/LICENSE.influx deleted file mode 100644 index f21351ced0..0000000000 --- a/licenses-binary/LICENSE.influx +++ /dev/null @@ -1,21 +0,0 @@ -The MIT License (MIT) - -Copyright (c) 2014-2017 Stefan Majer - -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.janino b/licenses-binary/LICENSE.janino deleted file mode 100644 index ef871e2426..0000000000 --- a/licenses-binary/LICENSE.janino +++ /dev/null @@ -1,31 +0,0 @@ -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 deleted file mode 100644 index e34763968c..0000000000 --- a/licenses-binary/LICENSE.jline +++ /dev/null @@ -1,7 +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 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 deleted file mode 100644 index e3dbacb999..0000000000 --- a/licenses-binary/LICENSE.jquery +++ /dev/null @@ -1,20 +0,0 @@ -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.jsr166y b/licenses-binary/LICENSE.jsr166y deleted file mode 100644 index b1c292b54c..0000000000 --- a/licenses-binary/LICENSE.jsr166y +++ /dev/null @@ -1,26 +0,0 @@ -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.jzlib b/licenses-binary/LICENSE.jzlib deleted file mode 100644 index 29ad562af0..0000000000 --- a/licenses-binary/LICENSE.jzlib +++ /dev/null @@ -1,26 +0,0 @@ -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 deleted file mode 100644 index e1cd88478e..0000000000 --- a/licenses-binary/LICENSE.kryo +++ /dev/null @@ -1,10 +0,0 @@ -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 deleted file mode 100644 index e3a30b4eaf..0000000000 --- a/licenses-binary/LICENSE.lodash +++ /dev/null @@ -1,10 +0,0 @@ -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.minlog b/licenses-binary/LICENSE.minlog deleted file mode 100644 index e1cd88478e..0000000000 --- a/licenses-binary/LICENSE.minlog +++ /dev/null @@ -1,10 +0,0 @@ -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 deleted file mode 100644 index 8618b7333d..0000000000 --- a/licenses-binary/LICENSE.moment +++ /dev/null @@ -1,22 +0,0 @@ -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 deleted file mode 100644 index 06ec6fbe07..0000000000 --- a/licenses-binary/LICENSE.moment-duration-format +++ /dev/null @@ -1,21 +0,0 @@ -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.monaco-editor b/licenses-binary/LICENSE.monaco-editor deleted file mode 100644 index 862172d90e..0000000000 --- a/licenses-binary/LICENSE.monaco-editor +++ /dev/null @@ -1,9 +0,0 @@ -The MIT License (MIT) - -Copyright (c) 2019 Microsoft Corporation - -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.ng-zorro-antd b/licenses-binary/LICENSE.ng-zorro-antd deleted file mode 100644 index 2dae87549e..0000000000 --- a/licenses-binary/LICENSE.ng-zorro-antd +++ /dev/null @@ -1,22 +0,0 @@ -MIT LICENSE - -Copyright (c) 2017-present Alibaba.com - -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.protobuf b/licenses-binary/LICENSE.protobuf deleted file mode 100644 index 19b305b000..0000000000 --- a/licenses-binary/LICENSE.protobuf +++ /dev/null @@ -1,32 +0,0 @@ -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.py4j b/licenses-binary/LICENSE.py4j deleted file mode 100644 index 0f45e3e464..0000000000 --- a/licenses-binary/LICENSE.py4j +++ /dev/null @@ -1,26 +0,0 @@ -Copyright (c) 2009-2018, Barthelemy Dagenais and individual 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. - -- The name of the author may not 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.pyrolite b/licenses-binary/LICENSE.pyrolite deleted file mode 100644 index ad923a6ea4..0000000000 --- a/licenses-binary/LICENSE.pyrolite +++ /dev/null @@ -1,21 +0,0 @@ -MIT License - -Copyright (c) by Irmen de Jong - -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.qtip2 b/licenses-binary/LICENSE.qtip2 deleted file mode 100644 index 0cfb790dc9..0000000000 --- a/licenses-binary/LICENSE.qtip2 +++ /dev/null @@ -1,22 +0,0 @@ -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.rxjs b/licenses-binary/LICENSE.rxjs deleted file mode 100644 index 65afc31116..0000000000 --- a/licenses-binary/LICENSE.rxjs +++ /dev/null @@ -1,201 +0,0 @@ - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. 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 and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) 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. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright (c) 2015-2018 Google, Inc., Netflix, Inc., Microsoft Corp. and contributors - - 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/licenses-binary/LICENSE.scala b/licenses-binary/LICENSE.scala deleted file mode 100644 index 6d8bdabbbe..0000000000 --- a/licenses-binary/LICENSE.scala +++ /dev/null @@ -1,11 +0,0 @@ -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 deleted file mode 100644 index 6d6a875878..0000000000 --- a/licenses-binary/LICENSE.scopt +++ /dev/null @@ -1,21 +0,0 @@ -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 deleted file mode 100644 index 93119e70ed..0000000000 --- a/licenses-binary/LICENSE.slf4j +++ /dev/null @@ -1,21 +0,0 @@ -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 deleted file mode 100644 index 93119e70ed..0000000000 --- a/licenses-binary/LICENSE.slf4j-api +++ /dev/null @@ -1,21 +0,0 @@ -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.split b/licenses-binary/LICENSE.split deleted file mode 100644 index 19b305b000..0000000000 --- a/licenses-binary/LICENSE.split +++ /dev/null @@ -1,32 +0,0 @@ -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.tinycolor2 b/licenses-binary/LICENSE.tinycolor2 deleted file mode 100644 index 38debb194c..0000000000 --- a/licenses-binary/LICENSE.tinycolor2 +++ /dev/null @@ -1,21 +0,0 @@ -MIT -Copyright (c), Brian Grinstead, http://briangrinstead.com - -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.tslib b/licenses-binary/LICENSE.tslib deleted file mode 100644 index dfcd5509ed..0000000000 --- a/licenses-binary/LICENSE.tslib +++ /dev/null @@ -1,57 +0,0 @@ -Apache-2.0 -Apache License - -Version 2.0, January 2004 - -http://www.apache.org/licenses/ - -TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - -1. Definitions. - -"License" shall mean the terms and conditions for use, reproduction, and distribution as defined by Sections 1 through 9 of this document. - -"Licensor" shall mean the copyright owner or entity authorized by the copyright owner that is granting the License. - -"Legal Entity" shall mean the union of the acting entity and all other entities that control, are controlled by, or are under common control with that entity. For the purposes of this definition, "control" means (i) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (ii) ownership of fifty percent (50%) or more of the outstanding shares, or (iii) beneficial ownership of such entity. - -"You" (or "Your") shall mean an individual or Legal Entity exercising permissions granted by this License. - -"Source" form shall mean the preferred form for making modifications, including but not limited to software source code, documentation source, and configuration files. - -"Object" form shall mean any form resulting from mechanical transformation or translation of a Source form, including but not limited to compiled object code, generated documentation, and conversions to other media types. - -"Work" shall mean the work of authorship, whether in Source or Object form, made available under the License, as indicated by a copyright notice that is included in or attached to the work (an example is provided in the Appendix below). - -"Derivative Works" shall mean any work, whether in Source or Object form, that is based on (or derived from) the Work and for which the editorial revisions, annotations, elaborations, or other modifications represent, as a whole, an original work of authorship. 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 and Derivative Works thereof. - -"Contribution" shall mean any work of authorship, including the original version of the Work and any modifications or additions to that Work or Derivative Works thereof, that is intentionally submitted to Licensor for inclusion in the Work by the copyright owner or by an individual or Legal Entity authorized to submit on behalf of the copyright owner. For the purposes of this definition, "submitted" means any form of electronic, verbal, or written communication sent to the Licensor or its representatives, including but not limited to communication on electronic mailing lists, source code control systems, and issue tracking systems that are managed by, or on behalf of, the Licensor for the purpose of discussing and improving the Work, but excluding communication that is conspicuously marked or otherwise designated in writing by the copyright owner as "Not a Contribution." - -"Contributor" shall mean Licensor and any individual or Legal Entity on behalf of whom a Contribution has been received by Licensor and subsequently incorporated within the Work. - -2. Grant of Copyright License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable copyright license to reproduce, prepare Derivative Works of, publicly display, publicly perform, sublicense, and distribute the Work and such Derivative Works in Source or Object form. - -3. Grant of Patent License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable (except as stated in this section) patent license to make, have made, use, offer to sell, sell, import, and otherwise transfer the Work, where such license applies only to those patent claims licensable by such Contributor that are necessarily infringed by their Contribution(s) alone or by combination of their Contribution(s) with the Work to which such Contribution(s) was submitted. If You institute patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Work or a Contribution incorporated within the Work constitutes direct or contributory patent infringement, then any patent licenses granted to You under this License for that Work shall terminate as of the date such litigation is filed. - -4. Redistribution. You may reproduce and distribute copies of the Work or Derivative Works thereof in any medium, with or without modifications, and in Source or Object form, provided that You meet the following conditions: - -You must give any other recipients of the Work or Derivative Works a copy of this License; and - -You must cause any modified files to carry prominent notices stating that You changed the files; and - -You must retain, in the Source form of any Derivative Works that You distribute, all copyright, patent, trademark, and attribution notices from the Source form of the Work, excluding those notices that do not pertain to any part of the Derivative Works; and - -If the Work includes a "NOTICE" text file as part of its distribution, then any Derivative Works that You distribute must include a readable copy of the attribution notices contained within such NOTICE file, excluding those notices that do not pertain to any part of the Derivative Works, in at least one of the following places: within a NOTICE text file distributed as part of the Derivative Works; within the Source form or documentation, if provided along with the Derivative Works; or, within a display generated by the Derivative Works, if and wherever such third-party notices normally appear. The contents of the NOTICE file are for informational purposes only and do not modify the License. You may add Your own attribution notices within Derivative Works that You distribute, alongside or as an addendum to the NOTICE text from the Work, provided that such additional attribution notices cannot be construed as modifying the License. You may add Your own copyright statement to Your modifications and may provide additional or different license terms and conditions for use, reproduction, or distribution of Your modifications, or for any such Derivative Works as a whole, provided Your use, reproduction, and distribution of the Work otherwise complies with the conditions stated in this License. - -5. Submission of Contributions. Unless You explicitly state otherwise, any Contribution intentionally submitted for inclusion in the Work by You to the Licensor shall be under the terms and conditions of this License, without any additional terms or conditions. Notwithstanding the above, nothing herein shall supersede or modify the terms of any separate license agreement you may have executed with Licensor regarding such Contributions. - -6. Trademarks. This License does not grant permission to use the trade names, trademarks, service marks, or product names of the Licensor, except as required for reasonable and customary use in describing the origin of the Work and reproducing the content of the NOTICE file. - -7. Disclaimer of Warranty. Unless required by applicable law or agreed to in writing, Licensor provides the Work (and each Contributor provides its Contributions) 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. You are solely responsible for determining the appropriateness of using or redistributing the Work and assume any risks associated with Your exercise of permissions under this License. - -8. Limitation of Liability. In no event and under no legal theory, whether in tort (including negligence), contract, or otherwise, unless required by applicable law (such as deliberate and grossly negligent acts) or agreed to in writing, shall any Contributor be liable to You for damages, including any direct, indirect, special, incidental, or consequential damages of any character arising as a result of this License or out of the use or inability to use the Work (including but not limited to damages for loss of goodwill, work stoppage, computer failure or malfunction, or any and all other commercial damages or losses), even if such Contributor has been advised of the possibility of such damages. - -9. Accepting Warranty or Additional Liability. While redistributing the Work or Derivative Works thereof, You may choose to offer, and charge a fee for, acceptance of support, warranty, indemnity, or other liability obligations and/or rights consistent with this License. However, in accepting such obligations, You may act only on Your own behalf and on Your sole responsibility, not on behalf of any other Contributor, and only if You agree to indemnify, defend, and hold each Contributor harmless for any liability incurred by, or claims asserted against, such Contributor by reason of your accepting any such warranty or additional liability. - -END OF TERMS AND CONDITIONS - diff --git a/licenses-binary/LICENSE.webbit b/licenses-binary/LICENSE.webbit deleted file mode 100644 index ec5f348998..0000000000 --- a/licenses-binary/LICENSE.webbit +++ /dev/null @@ -1,38 +0,0 @@ -(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.zone b/licenses-binary/LICENSE.zone deleted file mode 100644 index c5de4726c6..0000000000 --- a/licenses-binary/LICENSE.zone +++ /dev/null @@ -1,21 +0,0 @@ -The MIT License - -Copyright (c) 2016-2018 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. diff --git a/tools/travis_controller.sh b/tools/travis_controller.sh index 91392f658d..06d42ff667 100755 --- a/tools/travis_controller.sh +++ b/tools/travis_controller.sh @@ -126,39 +126,6 @@ if [ $STAGE == "$STAGE_COMPILE" ]; then echo "==============================================================================" fi - if [ $EXIT_CODE == 0 ]; then - if [[ $PROFILE == *"scala-2.11"* ]]; then - ./tools/releasing/collect_license_files.sh ./build-target - diff "NOTICE-binary" "licenses-output/NOTICE-binary" - EXIT_CODE=$(($EXIT_CODE+$?)) - diff -r "licenses-binary" "licenses-output/licenses-binary" - EXIT_CODE=$(($EXIT_CODE+$?)) - - if [ $EXIT_CODE != 0 ]; then - echo "==============================================================================" - echo "ERROR: binary licensing is out-of-date." - echo "Please update NOTICE-binary and licenses-binary:" - echo "Step 1: Rebuild flink" - echo "Step 2: Run 'tools/releasing/collect_license_files.sh build-target'" - echo " This extracts all the licensing files from the distribution, and puts them in 'licenses-output'." - echo " If the build-target symlink does not exist after building flink, point the tool to 'flink-dist/target/flink--bin/flink-' instead." - echo "Step 3: Replace existing licensing" - echo " Delete NOTICE-binary and the entire licenses-binary directory." - echo " Copy the contents in 'licenses-output' into the root directory of the Flink project." - echo "Step 4: Remember to commit the changes!" - echo "==============================================================================" - fi - else - echo "==============================================================================" - echo "Ignoring the license file check because built uses different Scala version than 2.11. See FLINK-14008." - echo "==============================================================================" - fi - else - echo "==============================================================================" - echo "Previous build failure detected, skipping licensing check." - echo "==============================================================================" - fi - if [ $EXIT_CODE == 0 ]; then echo "Creating cache build directory $CACHE_FLINK_DIR" mkdir -p "$CACHE_FLINK_DIR" -- Gitee From cc519d431305531a1e28754deda468c224435d15 Mon Sep 17 00:00:00 2001 From: Nick Pavlakis Date: Wed, 20 Mar 2019 15:04:23 -0700 Subject: [PATCH 264/268] [FLINK-5490] Verify that ExecutionEnvironment#getExecutionPlan() does not clear sinks --- .../client/ExecutionEnvironmentTest.java | 66 +++++++++++++++++++ 1 file changed, 66 insertions(+) create mode 100644 flink-clients/src/test/java/org/apache/flink/client/ExecutionEnvironmentTest.java diff --git a/flink-clients/src/test/java/org/apache/flink/client/ExecutionEnvironmentTest.java b/flink-clients/src/test/java/org/apache/flink/client/ExecutionEnvironmentTest.java new file mode 100644 index 0000000000..f512fe8ad8 --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/ExecutionEnvironmentTest.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.client; + +import org.apache.flink.api.common.functions.MapFunction; +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.util.TestLogger; + +import org.junit.Test; + +import java.io.Serializable; + +import static junit.framework.TestCase.fail; + + +/** + * Tests for {@link ExecutionEnvironment}. + * + *

    NOTE: This test is in the flink-client package because we cannot have it in flink-java, + * where the JSON plan generator is not available. Making it available, by depending on + * flink-optimizer would create a cyclic dependency. + */ +public class ExecutionEnvironmentTest extends TestLogger implements Serializable { + + /** + * Tests that verifies consecutive calls to {@link ExecutionEnvironment#getExecutionPlan()} do + * not cause any exceptions. {@link ExecutionEnvironment#getExecutionPlan()} must not modify + * the + * state of the plan + */ + @Test + public void testExecuteAfterGetExecutionPlanContextEnvironment() { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + DataSet baseSet = env.fromElements(1, 2); + + DataSet result = baseSet.map((MapFunction) value -> value * 2); + result.output(new DiscardingOutputFormat<>()); + + try { + env.getExecutionPlan(); + env.getExecutionPlan(); + } catch (Exception e) { + fail("Consecutive #getExecutionPlan calls caused an exception."); + } + } +} -- Gitee From 87f84e0c67f620b6b06cf83fcd775fc0ed2ede48 Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Thu, 17 Oct 2019 21:43:04 +0800 Subject: [PATCH 265/268] [hotfix][javadocs] Fix typo --- .../api/common/typeutils/CompositeTypeSerializerSnapshot.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerSnapshot.java index 825289664f..fc4a40d80a 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerSnapshot.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerSnapshot.java @@ -58,7 +58,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; * This means that the outer snapshot's version can be maintained only taking into account changes in how the * outer snapshot is written. Any changes in the base format does not require upticks in the outer snapshot's version. * - *

    Serialization Format + *

    Serialization Format

    * *

    The current version of the serialization format of a {@link CompositeTypeSerializerSnapshot} is as follows: * -- Gitee From cbf7b76923feec5ce8c9cb45a3082339be5ff17e Mon Sep 17 00:00:00 2001 From: Rong Rong Date: Sun, 29 Sep 2019 12:14:02 -0700 Subject: [PATCH 266/268] [FLINK-12399][table-planner] Check the table source digest changed after project/filter had been pushed down into it. Also fix table sources that incorrectly overrides explainSource without explaining pushdown This closes #8468 --- docs/dev/table/sourceSinks.md | 8 +- docs/dev/table/sourceSinks.zh.md | 8 +- .../flink/addons/hbase/HBaseTableSource.java | 6 +- .../org/apache/flink/orc/OrcTableSource.java | 9 +- .../formats/parquet/ParquetTableSource.java | 4 +- .../PushFilterIntoTableSourceScanRule.scala | 13 +- ...PushPartitionIntoTableSourceScanRule.scala | 8 +- .../PushProjectIntoTableSourceScanRule.scala | 16 ++- .../plan/batch/sql/TableSourceTest.xml | 36 +++--- .../PushFilterIntoTableSourceScanRuleTest.xml | 32 ++--- .../plan/stream/sql/TableSourceTest.xml | 36 +++--- .../planner/utils/testTableSources.scala | 3 +- .../PushFilterIntoTableSourceScanRule.scala | 8 ++ .../PushProjectIntoTableSourceScanRule.scala | 15 ++- .../flink/table/api/TableSourceTest.scala | 21 +++- .../TableSourceValidationTest.scala | 72 +++++++++++ .../utils/TestFilterableTableSource.scala | 118 +++++++++++++++--- .../flink/table/utils/testTableSources.scala | 62 ++++++++- 18 files changed, 379 insertions(+), 96 deletions(-) create mode 100644 flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/TableSourceValidationTest.scala diff --git a/docs/dev/table/sourceSinks.md b/docs/dev/table/sourceSinks.md index 01ebecaea3..8da8046ee4 100644 --- a/docs/dev/table/sourceSinks.md +++ b/docs/dev/table/sourceSinks.md @@ -263,6 +263,8 @@ ProjectableTableSource[T] { * `projectFields(fields)`: Returns a *copy* of the `TableSource` with adjusted physical return type. The `fields` parameter provides the indexes of the fields that must be provided by the `TableSource`. The indexes relate to the `TypeInformation` of the physical return type, *not* to the logical table schema. The copied `TableSource` must adjust its return type and the returned `DataStream` or `DataSet`. The `TableSchema` of the copied `TableSource` must not be changed, i.e, it must be the same as the original `TableSource`. If the `TableSource` implements the `DefinedFieldMapping` interface, the field mapping must be adjusted to the new return type. +Attention In order for Flink to distinguish a projection push-down table source from its original form, `explainSource` method must be override to include information regarding the projected fields. + The `ProjectableTableSource` adds support to project flat fields. If the `TableSource` defines a table with nested schema, it can implement the `NestedFieldsProjectableTableSource` to extend the projection to nested fields. The `NestedFieldsProjectableTableSource` is defined as follows:

    @@ -285,7 +287,9 @@ NestedFieldsProjectableTableSource[T] {
    -* `projectNestedField(fields, nestedFields)`: Returns a *copy* of the `TableSource` with adjusted physical return type. Fields of the physical return type may be removed or reordered but their type must not be changed. The contract of this method is essentially the same as for the `ProjectableTableSource.projectFields()` method. In addition, the `nestedFields` parameter contains for each field index in the `fields` list, a list of paths to all nested fields that are accessed by the query. All other nested fields do not need to be read, parsed, and set in the records that are produced by the `TableSource`. **IMPORTANT** the types of the projected fields must not be changed but unused fields may be set to null or to a default value. +* `projectNestedField(fields, nestedFields)`: Returns a *copy* of the `TableSource` with adjusted physical return type. Fields of the physical return type may be removed or reordered but their type must not be changed. The contract of this method is essentially the same as for the `ProjectableTableSource.projectFields()` method. In addition, the `nestedFields` parameter contains for each field index in the `fields` list, a list of paths to all nested fields that are accessed by the query. All other nested fields do not need to be read, parsed, and set in the records that are produced by the `TableSource`. + +Attention the types of the projected fields must not be changed but unused fields may be set to null or to a default value. {% top %} @@ -322,6 +326,8 @@ FilterableTableSource[T] { * `applyPredicate(predicates)`: Returns a *copy* of the `TableSource` with added predicates. The `predicates` parameter is a mutable list of conjunctive predicates that are "offered" to the `TableSource`. The `TableSource` accepts to evaluate a predicate by removing it from the list. Predicates that are left in the list will be evaluated by a subsequent filter operator. * `isFilterPushedDown()`: Returns true if the `applyPredicate()` method was called before. Hence, `isFilterPushedDown()` must return true for all `TableSource` instances returned from a `applyPredicate()` call. +Attention In order for Flink to distinguish a filter push-down table source from its original form, `explainSource` method must be override to include information regarding the push-down filters. + {% top %} ### Defining a TableSource for Lookups diff --git a/docs/dev/table/sourceSinks.zh.md b/docs/dev/table/sourceSinks.zh.md index 7fd872832d..82398a44d8 100644 --- a/docs/dev/table/sourceSinks.zh.md +++ b/docs/dev/table/sourceSinks.zh.md @@ -263,6 +263,8 @@ ProjectableTableSource[T] { * `projectFields(fields)`: Returns a *copy* of the `TableSource` with adjusted physical return type. The `fields` parameter provides the indexes of the fields that must be provided by the `TableSource`. The indexes relate to the `TypeInformation` of the physical return type, *not* to the logical table schema. The copied `TableSource` must adjust its return type and the returned `DataStream` or `DataSet`. The `TableSchema` of the copied `TableSource` must not be changed, i.e, it must be the same as the original `TableSource`. If the `TableSource` implements the `DefinedFieldMapping` interface, the field mapping must be adjusted to the new return type. +Attention In order for Flink to distinguish a projection push-down table source from its original form, `explainSource` method must be override to include information regarding the projected fields. + The `ProjectableTableSource` adds support to project flat fields. If the `TableSource` defines a table with nested schema, it can implement the `NestedFieldsProjectableTableSource` to extend the projection to nested fields. The `NestedFieldsProjectableTableSource` is defined as follows:
    @@ -285,7 +287,9 @@ NestedFieldsProjectableTableSource[T] {
    -* `projectNestedField(fields, nestedFields)`: Returns a *copy* of the `TableSource` with adjusted physical return type. Fields of the physical return type may be removed or reordered but their type must not be changed. The contract of this method is essentially the same as for the `ProjectableTableSource.projectFields()` method. In addition, the `nestedFields` parameter contains for each field index in the `fields` list, a list of paths to all nested fields that are accessed by the query. All other nested fields do not need to be read, parsed, and set in the records that are produced by the `TableSource`. **IMPORTANT** the types of the projected fields must not be changed but unused fields may be set to null or to a default value. +* `projectNestedField(fields, nestedFields)`: Returns a *copy* of the `TableSource` with adjusted physical return type. Fields of the physical return type may be removed or reordered but their type must not be changed. The contract of this method is essentially the same as for the `ProjectableTableSource.projectFields()` method. In addition, the `nestedFields` parameter contains for each field index in the `fields` list, a list of paths to all nested fields that are accessed by the query. All other nested fields do not need to be read, parsed, and set in the records that are produced by the `TableSource`. + +Attention the types of the projected fields must not be changed but unused fields may be set to null or to a default value. {% top %} @@ -322,6 +326,8 @@ FilterableTableSource[T] { * `applyPredicate(predicates)`: Returns a *copy* of the `TableSource` with added predicates. The `predicates` parameter is a mutable list of conjunctive predicates that are "offered" to the `TableSource`. The `TableSource` accepts to evaluate a predicate by removing it from the list. Predicates that are left in the list will be evaluated by a subsequent filter operator. * `isFilterPushedDown()`: Returns true if the `applyPredicate()` method was called before. Hence, `isFilterPushedDown()` must return true for all `TableSource` instances returned from a `applyPredicate()` call. +Attention In order for Flink to distinguish a filter push-down table source from its original form, `explainSource` method must be override to include information regarding the push-down filters. + {% top %} ### Defining a TableSource for Lookups diff --git a/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java b/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java index b1e716110e..98dfc62b34 100644 --- a/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java +++ b/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseTableSource.java @@ -31,12 +31,13 @@ import org.apache.flink.table.sources.BatchTableSource; import org.apache.flink.table.sources.LookupableTableSource; import org.apache.flink.table.sources.ProjectableTableSource; import org.apache.flink.table.sources.StreamTableSource; -import org.apache.flink.table.utils.TableConnectorUtils; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import org.apache.hadoop.conf.Configuration; +import java.util.Arrays; + /** * Creates a TableSource to scan an HBase table. * @@ -141,7 +142,8 @@ public class HBaseTableSource implements BatchTableSource, ProjectableTable @Override public String explainSource() { - return TableConnectorUtils.generateRuntimeName(this.getClass(), getTableSchema().getFieldNames()); + return "HBaseTableSource[schema=" + Arrays.toString(getTableSchema().getFieldNames()) + + ", projectFields=" + Arrays.toString(projectFields) + "]"; } @Override diff --git a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java index 6e3ada4c49..011e93da02 100644 --- a/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java +++ b/flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java @@ -213,14 +213,15 @@ public class OrcTableSource @Override public String explainSource() { - return "OrcFile[path=" + path + ", schema=" + orcSchema + ", filter=" + predicateString() + "]"; + return "OrcFile[path=" + path + ", schema=" + orcSchema + ", filter=" + predicateString() + + ", selectedFields=" + Arrays.toString(selectedFields) + "]"; } private String predicateString() { - if (predicates != null) { - return "AND(" + Arrays.toString(predicates) + ")"; - } else { + if (predicates == null || predicates.length == 0) { return "TRUE"; + } else { + return "AND(" + Arrays.toString(predicates) + ")"; } } diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetTableSource.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetTableSource.java index 0b5d168bd2..9144727973 100644 --- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetTableSource.java +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetTableSource.java @@ -66,6 +66,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; /** @@ -223,7 +224,8 @@ public class ParquetTableSource @Override public String explainSource() { return "ParquetFile[path=" + path + ", schema=" + parquetSchema + ", filter=" + predicateString() - + ", typeInfo=" + typeInfo + "]"; + + ", typeInfo=" + typeInfo + ", selectedFields=" + Arrays.toString(selectedFields) + + ", pushDownStatus=" + isFilterPushedDown + "]"; } private String predicateString() { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala index a4363e2650..a70c3ca8e7 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.rules.logical import org.apache.flink.table.api.config.OptimizerConfigOptions +import org.apache.flink.table.api.TableException import org.apache.flink.table.expressions.Expression import org.apache.flink.table.planner.calcite.FlinkContext import org.apache.flink.table.planner.expressions.converter.ExpressionConverter @@ -103,7 +104,17 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule( val remainingPredicates = new util.LinkedList[Expression]() predicates.foreach(e => remainingPredicates.add(e)) - val newRelOptTable = applyPredicate(remainingPredicates, relOptTable, relBuilder.getTypeFactory) + val newRelOptTable: FlinkRelOptTable = + applyPredicate(remainingPredicates, relOptTable, relBuilder.getTypeFactory) + val newTableSource = newRelOptTable.unwrap(classOf[TableSourceTable[_]]).tableSource + val oldTableSource = relOptTable.unwrap(classOf[TableSourceTable[_]]).tableSource + + if (newTableSource.asInstanceOf[FilterableTableSource[_]].isFilterPushedDown + && newTableSource.explainSource().equals(oldTableSource.explainSource)) { + throw new TableException("Failed to push filter into table source! " + + "table source with pushdown capability must override and change " + + "explainSource() API to explain the pushdown applied!") + } val newScan = new LogicalTableScan(scan.getCluster, scan.getTraitSet, newRelOptTable) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRule.scala index 1a9f8cd0e8..c752647bbf 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRule.scala @@ -23,12 +23,12 @@ import org.apache.flink.table.planner.plan.schema.{FlinkRelOptTable, TableSource import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, PartitionPruner, RexNodeExtractor} import org.apache.flink.table.sources.PartitionableTableSource - import org.apache.calcite.plan.RelOptRule.{none, operand} import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} import org.apache.calcite.rel.core.Filter import org.apache.calcite.rel.logical.LogicalTableScan import org.apache.calcite.rex.{RexInputRef, RexNode, RexShuttle} +import org.apache.flink.table.api.TableException import scala.collection.JavaConversions._ @@ -114,6 +114,12 @@ class PushPartitionIntoTableSourceScanRule extends RelOptRule( val newTableSource = tableSource.applyPartitionPruning(remainingPartitions) + if (newTableSource.explainSource().equals(tableSourceTable.tableSource.explainSource())) { + throw new TableException("Failed to push partition into table source! " + + "table source with pushdown capability must override and change " + + "explainSource() API to explain the pushdown applied!") + } + val statistic = tableSourceTable.statistic val newStatistic = if (remainingPartitions.size() == allPartitions.size()) { // Keep all Statistics if no predicates can be pushed down diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.scala index 6aad23c40e..e4973886ef 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.scala @@ -67,13 +67,23 @@ class PushProjectIntoTableSourceScanRule extends RelOptRule( val relOptTable = scan.getTable.asInstanceOf[FlinkRelOptTable] val tableSourceTable = relOptTable.unwrap(classOf[TableSourceTable[_]]) val oldTableSource = tableSourceTable.tableSource - val newTableSource = oldTableSource match { + val (newTableSource, isProjectSuccess) = oldTableSource match { case nested: NestedFieldsProjectableTableSource[_] => val nestedFields = RexNodeExtractor.extractRefNestedInputFields( project.getProjects, usedFields) - nested.projectNestedFields(usedFields, nestedFields) + (nested.projectNestedFields(usedFields, nestedFields), true) case projecting: ProjectableTableSource[_] => - projecting.projectFields(usedFields) + (projecting.projectFields(usedFields), true) + case nonProjecting: TableSource[_] => + // projection cannot be pushed to TableSource + (nonProjecting, false) + } + + if (isProjectSuccess + && newTableSource.explainSource().equals(oldTableSource.explainSource())) { + throw new TableException("Failed to push project into table source! " + + "table source with pushdown capability must override and change " + + "explainSource() API to explain the pushdown applied!") } // check that table schema of the new table source is identical to original diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml index 190f1c6151..ad8a1002e1 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml @@ -40,13 +40,13 @@ TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [Tes ($3, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> (price, 10)]) -+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable]], fields=[name, id, amount, price]) ++- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[]]]], fields=[name, id, amount, price]) ]]> @@ -58,13 +58,13 @@ Calc(select=[name, id, amount, price], where=[>(price, 10)]) ($2, 2), >($3, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> (amount, 2), >(price, 10))]) -+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable]], fields=[name, id, amount, price]) ++- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[]]]], fields=[name, id, amount, price]) ]]> @@ -76,13 +76,13 @@ Calc(select=[name, id, amount, price], where=[OR(>(amount, 2), >(price, 10))]) ($2, 2), <($2, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> (amount, 2), <(amount, 10))]) -+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable]], fields=[name, id, amount, price]) ++- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[]]]], fields=[name, id, amount, price]) ]]> @@ -94,12 +94,12 @@ Calc(select=[name, id, amount, price], where=[OR(>(amount, 2), <(amount, 10))]) ($2, 2), <($2, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> @@ -111,12 +111,12 @@ TableSourceScan(table=[[default_catalog, default_database, FilterableTable, sour ($2, 2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> @@ -128,13 +128,13 @@ TableSourceScan(table=[[default_catalog, default_database, FilterableTable, sour ($2, 2), >($3, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> (price, 10)]) -+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price]) ++- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price]) ]]> @@ -149,13 +149,13 @@ SELECT * FROM FilterableTable WHERE ($2, 2), <($1, 100), >(CAST($2):BIGINT, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> (CAST(amount), 10))]) -+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price]) ++- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price]) ]]> @@ -167,13 +167,13 @@ Calc(select=[name, id, amount, price], where=[AND(<(id, 100), >(CAST(amount), 10 ($2, 2), <(myUdf($2), 32))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> @@ -270,13 +270,13 @@ SELECT id FROM FilterableTable1 WHERE ($2, 14:25:02), >($1, 2017-02-03), >($3, 2017-02-03 14:25:02))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable1, source: [filterPushedDown=[false], filter=[]]]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.xml index 5a490fc2ed..5ecf131049 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.xml @@ -24,14 +24,14 @@ limitations under the License. ($3, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> ($3, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filterPushedDown=[true], filter=[]]]]) ]]> @@ -43,14 +43,14 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3]) ($2, 2), <($2, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> ($2, 2), <($2, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filterPushedDown=[true], filter=[]]]]) ]]> @@ -62,13 +62,13 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3]) ($2, 2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> @@ -80,13 +80,13 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3]) ($2, 2), <($2, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> @@ -98,14 +98,14 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3]) ($2, 2), >($3, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> ($3, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filter=[greaterThan(amount, 2)]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]]) ]]> @@ -117,14 +117,14 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3]) ($2, 2), >($3, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> ($2, 2), >($3, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filterPushedDown=[true], filter=[]]]]) ]]> @@ -139,14 +139,14 @@ SELECT * FROM MyTable WHERE ($2, 2), <($1, 100), >(CAST($2):BIGINT, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> (CAST($2):BIGINT, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filter=[greaterThan(amount, 2)]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]]) ]]> @@ -158,14 +158,14 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3]) ($2, 2), <(myUdf($2), 32))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml index 3d4f11f98c..837fa0829a 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml @@ -40,13 +40,13 @@ TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [Tes ($3, 10)]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> (price, 10)]) -+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable]], fields=[name, id, amount, price]) ++- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[]]]], fields=[name, id, amount, price]) ]]> @@ -58,13 +58,13 @@ Calc(select=[name, id, amount, price], where=[>(price, 10)]) ($2, 2), >($3, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> (amount, 2), >(price, 10))]) -+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable]], fields=[name, id, amount, price]) ++- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[]]]], fields=[name, id, amount, price]) ]]> @@ -76,13 +76,13 @@ Calc(select=[name, id, amount, price], where=[OR(>(amount, 2), >(price, 10))]) ($2, 2), <($2, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> (amount, 2), <(amount, 10))]) -+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable]], fields=[name, id, amount, price]) ++- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[]]]], fields=[name, id, amount, price]) ]]> @@ -94,12 +94,12 @@ Calc(select=[name, id, amount, price], where=[OR(>(amount, 2), <(amount, 10))]) ($2, 2)]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> @@ -140,12 +140,12 @@ Calc(select=[name, w$end AS EXPR$1, EXPR$2]) ($2, 2), <($2, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> @@ -157,13 +157,13 @@ TableSourceScan(table=[[default_catalog, default_database, FilterableTable, sour ($2, 2), >($3, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> (price, 10)]) -+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price]) ++- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price]) ]]> @@ -178,13 +178,13 @@ SELECT * FROM FilterableTable WHERE ($2, 2), <($1, 100), >(CAST($2):BIGINT, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> (CAST(amount), 10))]) -+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price]) ++- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price]) ]]> @@ -196,13 +196,13 @@ Calc(select=[name, id, amount, price], where=[AND(<(id, 100), >(CAST(amount), 10 ($2, 2), <(myUdf($2), 32))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[false], filter=[]]]]) ]]> @@ -383,13 +383,13 @@ SELECT id FROM FilterableTable1 WHERE ($2, 14:25:02), >($1, 2017-02-03), >($3, 2017-02-03 14:25:02))]) - +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable1]]) + +- LogicalTableScan(table=[[default_catalog, default_database, FilterableTable1, source: [filterPushedDown=[false], filter=[]]]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/testTableSources.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/testTableSources.scala index 24fab4251f..0416218eb8 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/testTableSources.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/testTableSources.scala @@ -364,9 +364,10 @@ class TestFilterableTableSource( override def explainSource(): String = { if (filterPredicates.nonEmpty) { + s"filterPushedDown=[$filterPushedDown], " + s"filter=[${filterPredicates.reduce((l, r) => unresolvedCall(AND, l, r)).toString}]" } else { - "" + s"filterPushedDown=[$filterPushedDown], filter=[]" } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala index 9c16135065..de1d0e00dd 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala @@ -23,6 +23,7 @@ import java.util import org.apache.calcite.plan.RelOptRule.{none, operand} import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} import org.apache.calcite.rex.RexProgram +import org.apache.flink.table.api.TableException import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog} import org.apache.flink.table.expressions.{Expression, PlannerExpression} import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalTableSourceScan} @@ -83,6 +84,13 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule( val newTableSource = filterableSource.applyPredicate(remainingPredicates) + if (newTableSource.asInstanceOf[FilterableTableSource[_]].isFilterPushedDown + && newTableSource.explainSource().equals(scan.tableSource.explainSource())) { + throw new TableException("Failed to push filter into table source! " + + "table source with pushdown capability must override and change " + + "explainSource() API to explain the pushdown applied!") + } + // check whether framework still need to do a filter val relBuilder = call.builder() val remainingCondition = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRule.scala index 3ea97abdce..9864cd0929 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRule.scala @@ -50,16 +50,23 @@ class PushProjectIntoTableSourceScanRule extends RelOptRule( if (!(0 until scan.getRowType.getFieldCount).toArray.sameElements(accessedLogicalFields)) { // try to push projection of physical fields to TableSource - val newTableSource = source match { + val (newTableSource, isProjectSuccess) = source match { case nested: NestedFieldsProjectableTableSource[_] => val nestedFields = RexProgramExtractor .extractRefNestedInputFields(calc.getProgram, accessedPhysicalFields) - nested.projectNestedFields(accessedPhysicalFields, nestedFields) + (nested.projectNestedFields(accessedPhysicalFields, nestedFields), true) case projecting: ProjectableTableSource[_] => - projecting.projectFields(accessedPhysicalFields) + (projecting.projectFields(accessedPhysicalFields), true) case nonProjecting: TableSource[_] => // projection cannot be pushed to TableSource - nonProjecting + (nonProjecting, false) + } + + if (isProjectSuccess + && newTableSource.explainSource().equals(scan.tableSource.explainSource())) { + throw new TableException("Failed to push project into table source! " + + "table source with pushdown capability must override and change " + + "explainSource() API to explain the pushdown applied!") } // check that table schema of the new table source is identical to original diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala index 5ae51ab5a3..5b7736442e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala @@ -55,10 +55,12 @@ class TableSourceTest extends TableTestBase { batchFilterableSourceTableNode( "table1", Array("name", "id", "amount", "price"), + isPushedDown = true, "'amount > 2"), batchFilterableSourceTableNode( "table2", Array("name", "id", "amount", "price"), + isPushedDown = true, "'amount > 2"), term("all", "true"), term("union", "name, id, amount, price") @@ -161,8 +163,11 @@ class TableSourceTest extends TableTestBase { val expected = unaryNode( "DataSetCalc", - "BatchTableSourceScan(table=[[default_catalog, default_database, filterableTable]], " + - "fields=[price, id, amount])", + batchFilterableSourceTableNode( + tableName, + Array("price", "id", "amount"), + isPushedDown = true, + ""), term("select", "price", "id", "amount"), term("where", "<(*(price, 2), 32)") ) @@ -188,6 +193,7 @@ class TableSourceTest extends TableTestBase { batchFilterableSourceTableNode( tableName, Array("price", "name", "amount"), + isPushedDown = true, "'amount > 2"), term("select", "price", "LOWER(name) AS _c1", "amount"), term("where", "<(*(price, 2), 32)") @@ -211,6 +217,7 @@ class TableSourceTest extends TableTestBase { val expected = batchFilterableSourceTableNode( tableName, Array("price", "id", "amount"), + isPushedDown = true, "'amount > 2 && 'amount < 32") util.verifyTable(result, expected) } @@ -234,6 +241,7 @@ class TableSourceTest extends TableTestBase { batchFilterableSourceTableNode( tableName, Array("price", "id", "amount"), + isPushedDown = true, "'amount > 2"), term("select", "price", "id", "amount"), term("where", "AND(<(id, 1.2E0:DOUBLE), OR(<(amount, 32), >(CAST(amount), 10)))") @@ -261,6 +269,7 @@ class TableSourceTest extends TableTestBase { batchFilterableSourceTableNode( tableName, Array("price", "id", "amount"), + isPushedDown = true, "'amount > 2"), term("select", "price", "id", "amount"), term("where", s"<(${Func0.getClass.getSimpleName}(amount), 32)") @@ -344,6 +353,7 @@ class TableSourceTest extends TableTestBase { streamFilterableSourceTableNode( tableName, Array("price", "id", "amount"), + isPushedDown = true, "'amount > 2"), term("select", "price", "id", "amount"), term("where", "<(*(price, 2), 32)") @@ -446,6 +456,7 @@ class TableSourceTest extends TableTestBase { val expected = batchFilterableSourceTableNode( tableName, Array("id"), + isPushedDown = true, expectedFilter ) util.verifyTable(result, expected) @@ -501,25 +512,27 @@ class TableSourceTest extends TableTestBase { def batchFilterableSourceTableNode( sourceName: String, fields: Array[String], + isPushedDown: Boolean, exp: String) : String = { "BatchTableSourceScan(" + s"table=[[default_catalog, default_database, $sourceName]], fields=[${ fields .mkString(", ") - }], source=[filter=[$exp]])" + }], source=[filterPushedDown=[$isPushedDown], filter=[$exp]])" } def streamFilterableSourceTableNode( sourceName: String, fields: Array[String], + isPushedDown: Boolean, exp: String) : String = { "StreamTableSourceScan(" + s"table=[[default_catalog, default_database, $sourceName]], fields=[${ fields .mkString(", ") - }], source=[filter=[$exp]])" + }], source=[filterPushedDown=[$isPushedDown], filter=[$exp]])" } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/TableSourceValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/TableSourceValidationTest.scala new file mode 100644 index 0000000000..11bdf6ebb8 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/TableSourceValidationTest.scala @@ -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.table.api.stream.table.validation + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.{TableException, TableSchema, Types} +import org.apache.flink.table.utils.{TableTestBase, TestFilterableTableSourceWithoutExplainSourceOverride, TestProjectableTableSourceWithoutExplainSourceOverride} +import org.hamcrest.Matchers +import org.junit.Test + +class TableSourceValidationTest extends TableTestBase { + + @Test + def testPushProjectTableSourceWithoutExplainSource(): Unit = { + expectedException.expectCause(Matchers.isA(classOf[TableException])) + + val tableSchema = new TableSchema( + Array("id", "rtime", "val", "ptime", "name"), + Array(Types.INT, Types.SQL_TIMESTAMP, Types.LONG, Types.SQL_TIMESTAMP, Types.STRING)) + val returnType = new RowTypeInfo( + Array(Types.INT, Types.STRING, Types.LONG, Types.LONG) + .asInstanceOf[Array[TypeInformation[_]]], + Array("id", "name", "val", "rtime")) + + val util = streamTestUtil() + util.tableEnv.registerTableSource( + "T", + new TestProjectableTableSourceWithoutExplainSourceOverride( + tableSchema, returnType, Seq(), "rtime", "ptime")) + + val t = util.tableEnv.scan("T").select('name, 'val, 'id) + + // must fail since pushed projection is not explained in source + util.explain(t) + } + + @Test + def testPushFilterableTableSourceWithoutExplainSource(): Unit = { + expectedException.expectCause(Matchers.isA(classOf[TableException])) + + val tableSource = TestFilterableTableSourceWithoutExplainSourceOverride() + val util = batchTestUtil() + + util.tableEnv.registerTableSource("T", tableSource) + + val t = util.tableEnv + .scan("T") + .select('price, 'id, 'amount) + .where("price * 2 < 32") + + // must fail since pushed filter is not explained in source + util.explain(t) + } +} diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala index 4f767f6994..79f5f32978 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala @@ -80,6 +80,53 @@ object TestFilterableTableSource { } } +object TestFilterableTableSourceWithoutExplainSourceOverride{ + + /** + * @return The default filterable table source. + */ + def apply(): TestFilterableTableSourceWithoutExplainSourceOverride = { + apply(defaultTypeInfo, defaultRows, defaultFilterableFields) + } + + /** + * A filterable data source with custom data. + * @param rowTypeInfo The type of the data. Its expected that both types and field + * names are provided. + * @param rows The data as a sequence of rows. + * @param filterableFields The fields that are allowed to be filtered on. + * @return The table source. + */ + def apply( + rowTypeInfo: RowTypeInfo, + rows: Seq[Row], + filterableFields: Set[String]) + : TestFilterableTableSourceWithoutExplainSourceOverride = { + new TestFilterableTableSourceWithoutExplainSourceOverride(rowTypeInfo, rows, filterableFields) + } + + private lazy val defaultFilterableFields = Set("amount") + + private lazy val defaultTypeInfo: RowTypeInfo = { + val fieldNames: Array[String] = Array("name", "id", "amount", "price") + val fieldTypes: Array[TypeInformation[_]] = Array(STRING, LONG, INT, DOUBLE) + new RowTypeInfo(fieldTypes, fieldNames) + } + + private lazy val defaultRows: Seq[Row] = { + for { + cnt <- 0 until 33 + } yield { + Row.of( + s"Record_$cnt", + cnt.toLong.asInstanceOf[AnyRef], + cnt.toInt.asInstanceOf[AnyRef], + cnt.toDouble.asInstanceOf[AnyRef]) + } + } +} + + /** * A data source that implements some very basic filtering in-memory in order to test * expression push-down logic. @@ -91,6 +138,53 @@ object TestFilterableTableSource { * @param filterPushedDown Whether predicates have been pushed down yet. */ class TestFilterableTableSource( + rowTypeInfo: RowTypeInfo, + data: Seq[Row], + filterableFields: Set[String] = Set(), + filterPredicates: Seq[Expression] = Seq(), + filterPushedDown: Boolean = false) + extends TestFilterableTableSourceWithoutExplainSourceOverride( + rowTypeInfo, + data, + filterableFields, + filterPredicates, + filterPushedDown + ) { + + override def applyPredicate(predicates: JList[Expression]): TableSource[Row] = { + val predicatesToUse = new mutable.ListBuffer[Expression]() + val iterator = predicates.iterator() + while (iterator.hasNext) { + val expr = iterator.next() + if (shouldPushDown(expr)) { + predicatesToUse += expr + iterator.remove() + } + } + + new TestFilterableTableSource( + rowTypeInfo, + data, + filterableFields, + predicatesToUse, + filterPushedDown = true) + } + + override def explainSource(): String = { + if (filterPredicates.nonEmpty) { + // TODO we cast to planner expression as a temporary solution to keep the old interfaces + s"filterPushedDown=[$filterPushedDown], filter=[${filterPredicates.reduce((l, r) => + And(l.asInstanceOf[PlannerExpression], r.asInstanceOf[PlannerExpression])).toString}]" + } else { + s"filterPushedDown=[$filterPushedDown], filter=[]" + } + } +} + +/** + * A [[TestFilterableTableSource]] without explain source override. + */ +class TestFilterableTableSourceWithoutExplainSourceOverride( rowTypeInfo: RowTypeInfo, data: Seq[Row], filterableFields: Set[String] = Set(), @@ -115,16 +209,6 @@ class TestFilterableTableSource( execEnv.fromCollection[Row](applyPredicatesToRows(data).asJava, getReturnType) } - override def explainSource(): String = { - if (filterPredicates.nonEmpty) { - // TODO we cast to planner expression as a temporary solution to keep the old interfaces - s"filter=[${filterPredicates.reduce((l, r) => - And(l.asInstanceOf[PlannerExpression], r.asInstanceOf[PlannerExpression])).toString}]" - } else { - "" - } - } - override def getReturnType: TypeInformation[Row] = rowTypeInfo override def applyPredicate(predicates: JList[Expression]): TableSource[Row] = { @@ -138,7 +222,7 @@ class TestFilterableTableSource( } } - new TestFilterableTableSource( + new TestFilterableTableSourceWithoutExplainSourceOverride( rowTypeInfo, data, filterableFields, @@ -148,18 +232,18 @@ class TestFilterableTableSource( override def isFilterPushedDown: Boolean = filterPushedDown - private def applyPredicatesToRows(rows: Seq[Row]): Seq[Row] = { + private[flink] def applyPredicatesToRows(rows: Seq[Row]): Seq[Row] = { rows.filter(shouldKeep) } - private def shouldPushDown(expr: Expression): Boolean = { + private[flink] def shouldPushDown(expr: Expression): Boolean = { expr match { case binExpr: BinaryComparison => shouldPushDown(binExpr) case _ => false } } - private def shouldPushDown(expr: BinaryComparison): Boolean = { + private[flink] def shouldPushDown(expr: BinaryComparison): Boolean = { (expr.left, expr.right) match { case (f: PlannerResolvedFieldReference, v: Literal) => filterableFields.contains(f.name) @@ -171,14 +255,14 @@ class TestFilterableTableSource( } } - private def shouldKeep(row: Row): Boolean = { + private[flink] def shouldKeep(row: Row): Boolean = { filterPredicates.isEmpty || filterPredicates.forall { case expr: BinaryComparison => binaryFilterApplies(expr, row) case expr => throw new RuntimeException(expr + " not supported!") } } - private def binaryFilterApplies(expr: BinaryComparison, row: Row): Boolean = { + private[flink] def binaryFilterApplies(expr: BinaryComparison, row: Row): Boolean = { val (lhsValue, rhsValue) = extractValues(expr, row) expr match { @@ -197,7 +281,7 @@ class TestFilterableTableSource( } } - private def extractValues(expr: BinaryComparison, row: Row) + private[flink] def extractValues(expr: BinaryComparison, row: Row) : (Comparable[Any], Comparable[Any]) = { (expr.left, expr.right) match { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/testTableSources.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/testTableSources.scala index 6f66158759..6a6d691bd8 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/testTableSources.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/testTableSources.scala @@ -94,15 +94,13 @@ class TestProjectableTableSource( rowtime: String = null, proctime: String = null, fieldMapping: Map[String, String] = null) - extends TestTableSourceWithTime[Row]( + extends TestProjectableTableSourceWithoutExplainSourceOverride( tableSchema, returnType, values, rowtime, proctime, - fieldMapping) - with ProjectableTableSource[Row] { - + fieldMapping) { override def projectFields(fields: Array[Int]): TableSource[Row] = { val rowType = returnType.asInstanceOf[RowTypeInfo] @@ -148,6 +146,62 @@ class TestProjectableTableSource( } } +class TestProjectableTableSourceWithoutExplainSourceOverride( + tableSchema: TableSchema, + returnType: TypeInformation[Row], + values: Seq[Row], + rowtime: String = null, + proctime: String = null, + fieldMapping: Map[String, String] = null) + extends TestTableSourceWithTime[Row]( + tableSchema, + returnType, + values, + rowtime, + proctime, + fieldMapping) + with ProjectableTableSource[Row] { + + override def projectFields(fields: Array[Int]): TableSource[Row] = { + + val rowType = returnType.asInstanceOf[RowTypeInfo] + + val (projectedNames: Array[String], projectedMapping) = if (fieldMapping == null) { + val projectedNames = fields.map(rowType.getFieldNames.apply(_)) + (projectedNames, null) + } else { + val invertedMapping = fieldMapping.map(_.swap) + val projectedNames = fields.map(rowType.getFieldNames.apply(_)) + + val projectedMapping: Map[String, String] = projectedNames.map{ f => + val logField = invertedMapping(f) + logField -> s"remapped-$f" + }.toMap + val renamedNames = projectedNames.map(f => s"remapped-$f") + (renamedNames, projectedMapping) + } + + val projectedTypes = fields.map(rowType.getFieldTypes.apply(_)) + val projectedReturnType = new RowTypeInfo( + projectedTypes.asInstanceOf[Array[TypeInformation[_]]], + projectedNames) + + val projectedValues = values.map { fromRow => + val pRow = new Row(fields.length) + fields.zipWithIndex.foreach{ case (from, to) => pRow.setField(to, fromRow.getField(from)) } + pRow + } + + new TestProjectableTableSourceWithoutExplainSourceOverride( + tableSchema, + projectedReturnType, + projectedValues, + rowtime, + proctime, + projectedMapping) + } +} + class TestNestedProjectableTableSource( tableSchema: TableSchema, returnType: TypeInformation[Row], -- Gitee From 4491165c6542ce1dd68562cb193a71f3a15bb96e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 18:18:40 +0200 Subject: [PATCH 267/268] [FLINK-14278] Extend DispatcherResourceManagerComponentFactory.create to take ioExecutor This closes #9831. --- .../runtime/dispatcher/runner/DispatcherRunnerFactory.java | 7 ++++++- .../dispatcher/runner/DispatcherRunnerFactoryImpl.java | 3 +++ .../apache/flink/runtime/entrypoint/ClusterEntrypoint.java | 1 + .../DefaultDispatcherResourceManagerComponentFactory.java | 3 +++ .../DispatcherResourceManagerComponentFactory.java | 3 +++ .../org/apache/flink/runtime/minicluster/MiniCluster.java | 5 +++++ .../flink/runtime/minicluster/TestingMiniCluster.java | 1 + .../flink/test/recovery/ProcessFailureCancelingITCase.java | 5 ++++- 8 files changed, 26 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.java index 9212ea871f..9cf8232563 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.java @@ -21,10 +21,15 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.rpc.RpcService; +import java.util.concurrent.Executor; + /** * Factory interface for the {@link DispatcherRunner}. */ public interface DispatcherRunnerFactory { - DispatcherRunner createDispatcherRunner(RpcService rpcService, PartialDispatcherServices partialDispatcherServices) throws Exception; + DispatcherRunner createDispatcherRunner( + RpcService rpcService, + Executor ioExecutor, + PartialDispatcherServices partialDispatcherServices) throws Exception; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactoryImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactoryImpl.java index e0926217ea..dc2654714a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactoryImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactoryImpl.java @@ -23,6 +23,8 @@ import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; import org.apache.flink.runtime.rpc.RpcService; +import java.util.concurrent.Executor; + /** * Factory which creates a {@link DispatcherRunnerImpl} which runs a {@link StandaloneDispatcher}. */ @@ -37,6 +39,7 @@ public class DispatcherRunnerFactoryImpl implements DispatcherRunnerFactory { @Override public DispatcherRunnerImpl createDispatcherRunner( RpcService rpcService, + Executor ioExecutor, PartialDispatcherServices partialDispatcherServices) throws Exception { return new DispatcherRunnerImpl(dispatcherFactory, rpcService, partialDispatcherServices); } 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 a50cc1be90..68bb95bb52 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 @@ -214,6 +214,7 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro clusterComponent = dispatcherResourceManagerComponentFactory.create( configuration, + ioExecutor, commonRpcService, haServices, blobServer, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java index dd89a757a6..9b72997cdc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java @@ -71,6 +71,7 @@ import javax.annotation.Nonnull; import java.util.ArrayList; import java.util.Collection; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; /** @@ -101,6 +102,7 @@ public class DefaultDispatcherResourceManagerComponentFactory implements Dispatc @Override public DispatcherResourceManagerComponent create( Configuration configuration, + Executor ioExecutor, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, BlobServer blobServer, @@ -199,6 +201,7 @@ public class DefaultDispatcherResourceManagerComponentFactory implements Dispatc log.debug("Starting Dispatcher."); dispatcherRunner = dispatcherRunnerFactory.createDispatcherRunner( rpcService, + ioExecutor, partialDispatcherServices); log.debug("Starting ResourceManager."); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponentFactory.java index 744941ca3a..752291d730 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponentFactory.java @@ -28,6 +28,8 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever; +import java.util.concurrent.Executor; + /** * Factory for the {@link DispatcherResourceManagerComponent}. */ @@ -35,6 +37,7 @@ public interface DispatcherResourceManagerComponentFactory { DispatcherResourceManagerComponent create( Configuration configuration, + Executor ioExecutor, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, BlobServer blobServer, 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 466269c109..7da4c6ddf2 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 @@ -227,6 +227,10 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { } } + protected Executor getIOExecutor() { + return ioExecutor; + } + @VisibleForTesting @Nonnull protected Collection getDispatcherResourceManagerComponents() { @@ -388,6 +392,7 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { return Collections.singleton( dispatcherResourceManagerComponentFactory.create( configuration, + ioExecutor, rpcServiceFactory.createRpcService(), haServices, blobServer, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/TestingMiniCluster.java b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/TestingMiniCluster.java index a4bb3dcd51..44c2d94779 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/TestingMiniCluster.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/TestingMiniCluster.java @@ -116,6 +116,7 @@ public class TestingMiniCluster extends MiniCluster { result.add( dispatcherResourceManagerComponentFactory.create( configuration, + getIOExecutor(), rpcServiceFactory.createRpcService(), haServices, blobServer, 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 bd76b0942a..db0b1be177 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 @@ -75,6 +75,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Map; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath; @@ -127,9 +128,10 @@ public class ProcessFailureCancelingITCase extends TestLogger { StandaloneResourceManagerFactory.INSTANCE); DispatcherResourceManagerComponent dispatcherResourceManagerComponent = null; + final ScheduledExecutorService ioExecutor = TestingUtils.defaultExecutor(); final HighAvailabilityServices haServices = HighAvailabilityServicesUtils.createHighAvailabilityServices( config, - TestingUtils.defaultExecutor(), + ioExecutor, HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION); try { @@ -143,6 +145,7 @@ public class ProcessFailureCancelingITCase extends TestLogger { dispatcherResourceManagerComponent = resourceManagerComponentFactory.create( config, + ioExecutor, rpcService, haServices, blobServerResource.getBlobServer(), -- Gitee From 34a1f9412ad126c3b60e851769022906da5e5a00 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 17 Oct 2019 19:22:32 +0200 Subject: [PATCH 268/268] [hotfix] Fix scheduler-ng profile to set system property jobmanager.scheduler=ng --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7eab0bdb76..4b49170c0f 100644 --- a/pom.xml +++ b/pom.xml @@ -644,7 +644,7 @@ under the License. scheduler-ng - ng + ng org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG -- Gitee

Note: the future is created upon calling {@link #isLeader(UUID)}. */ - public synchronized CompletableFuture getConfirmationFuture() { + public synchronized CompletableFuture getConfirmationFuture() { return confirmationFuture; } @@ -69,7 +71,7 @@ public class TestingLeaderElectionService implements LeaderElectionService { @Override public synchronized void confirmLeadership(UUID leaderSessionID, String leaderAddress) { if (confirmationFuture != null) { - confirmationFuture.complete(leaderSessionID); + confirmationFuture.complete(new LeaderConnectionInfo(leaderSessionID, leaderAddress)); } } @@ -90,7 +92,7 @@ public class TestingLeaderElectionService implements LeaderElectionService { contender.grantLeadership(leaderSessionID); } - return confirmationFuture; + return confirmationFuture.thenApply(LeaderConnectionInfo::getLeaderSessionId); } public synchronized void notLeader() { @@ -102,8 +104,8 @@ public class TestingLeaderElectionService implements LeaderElectionService { } public synchronized String getAddress() { - if (contender != null) { - return contender.getAddress(); + if (confirmationFuture.isDone()) { + return confirmationFuture.join().getAddress(); } else { throw new IllegalStateException("TestingLeaderElectionService has not been started."); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java index b476896b5d..bf5b2c8d3d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java @@ -43,6 +43,8 @@ import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; + import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.ObjectOutputStream; @@ -175,7 +177,7 @@ public class ZooKeeperLeaderElectionTest extends TestLogger { for (int i = 0; i < num; i++) { leaderElectionService[i] = ZooKeeperUtils.createLeaderElectionService(client, configuration); - contenders[i] = new TestingContender(TEST_URL + "_" + i, leaderElectionService[i]); + contenders[i] = new TestingContender(createAddress(i), leaderElectionService[i]); LOG.debug("Start leader election service for contender #{}.", i); @@ -199,7 +201,7 @@ public class ZooKeeperLeaderElectionTest extends TestLogger { TestingContender contender = contenders[index]; // check that the retrieval service has retrieved the correct leader - if (address.equals(contender.getAddress()) && listener.getLeaderSessionID().equals(contender.getLeaderSessionID())) { + if (address.equals(createAddress(index)) && listener.getLeaderSessionID().equals(contender.getLeaderSessionID())) { // kill the election service of the leader LOG.debug("Stop leader election service of contender #{}.", numberSeenLeaders); leaderElectionService[index].stop(); @@ -228,6 +230,11 @@ public class ZooKeeperLeaderElectionTest extends TestLogger { } } + @Nonnull + private String createAddress(int i) { + return TEST_URL + "_" + i; + } + /** * Tests the repeated reelection of {@link LeaderContender} once the current leader dies. * Furthermore, it tests that new LeaderElectionServices can be started later on and that they @@ -367,7 +374,7 @@ public class ZooKeeperLeaderElectionTest extends TestLogger { } assertEquals(listener2.getLeaderSessionID(), contender.getLeaderSessionID()); - assertEquals(listener2.getAddress(), contender.getAddress()); + assertEquals(listener2.getAddress(), TEST_URL); } finally { if (leaderElectionService != null) { diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java index 0d09f87902..ab4f70a482 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServicesTest.java @@ -49,7 +49,6 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.timeout; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; /** * Tests for YarnIntraNonHaMasterServices. @@ -150,8 +149,6 @@ public class YarnIntraNonHaMasterServicesTest extends TestLogger { private static LeaderContender mockContender(final LeaderElectionService service, final String address) { LeaderContender mockContender = mock(LeaderContender.class); - when(mockContender.getAddress()).thenReturn(address); - doAnswer(new Answer() { @Override public Void answer(InvocationOnMock invocation) throws Throwable { -- Gitee From 394d93036b7206aa049fd69c9d2daa938db642a2 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 13 Sep 2019 16:37:48 +0200 Subject: [PATCH 172/268] [hotfix] Add FutureUtils.completedVoidFuture --- .../apache/flink/runtime/concurrent/FutureUtils.java | 11 +++++++++++ 1 file changed, 11 insertions(+) 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 73f3ecb663..b60ee3cff1 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 @@ -64,6 +64,17 @@ import static org.apache.flink.util.Preconditions.checkNotNull; */ public class FutureUtils { + private static final CompletableFuture COMPLETED_VOID_FUTURE = CompletableFuture.completedFuture(null); + + /** + * Returns a completed future of type {@link Void}. + * + * @return a completed future of type {@link Void} + */ + public static CompletableFuture completedVoidFuture() { + return COMPLETED_VOID_FUTURE; + } + // ------------------------------------------------------------------------ // retrying operations // ------------------------------------------------------------------------ -- Gitee From c2f686637eed86ff02aec08a16c3fcca09f352fe Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 17:15:16 +0200 Subject: [PATCH 173/268] [hotfix] Use simulated self-type idiom in TestingRestful/DispatcherGateway.Builder --- .../LeaderRetrievalHandlerTest.java | 2 +- .../handlers/JarDeleteHandlerTest.java | 2 +- .../runtime/rest/RestServerSSLAuthITCase.java | 2 +- .../rest/handler/AbstractHandlerTest.java | 2 +- .../job/JobExecutionResultHandlerTest.java | 8 +- .../job/JobVertexBackPressureHandlerTest.java | 2 +- .../legacy/ExecutionGraphCacheTest.java | 2 +- .../webmonitor/TestingDispatcherGateway.java | 9 ++- .../webmonitor/TestingRestfulGateway.java | 80 +++++++++++-------- 9 files changed, 64 insertions(+), 45 deletions(-) diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/LeaderRetrievalHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/LeaderRetrievalHandlerTest.java index a99daf7730..67acfe4112 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/LeaderRetrievalHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/LeaderRetrievalHandlerTest.java @@ -66,7 +66,7 @@ public class LeaderRetrievalHandlerTest extends TestLogger { final Time timeout = Time.seconds(10L); final CompletableFuture gatewayFuture = new CompletableFuture<>(); final GatewayRetriever gatewayRetriever = () -> gatewayFuture; - final RestfulGateway gateway = TestingRestfulGateway.newBuilder().build(); + final RestfulGateway gateway = new TestingRestfulGateway.Builder().build(); final TestingHandler testingHandler = new TestingHandler( gatewayRetriever, diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandlerTest.java index fced9010ee..edbeca4bbe 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandlerTest.java @@ -70,7 +70,7 @@ public class JarDeleteHandlerTest extends TestLogger { @Before public void setUp() throws Exception { jarDir = temporaryFolder.newFolder().toPath(); - restfulGateway = TestingRestfulGateway.newBuilder().build(); + restfulGateway = new TestingRestfulGateway.Builder().build(); jarDeleteHandler = new JarDeleteHandler( () -> CompletableFuture.completedFuture(restfulGateway), Time.seconds(10), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerSSLAuthITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerSSLAuthITCase.java index eecf7b0d08..8f51a41740 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerSSLAuthITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerSSLAuthITCase.java @@ -89,7 +89,7 @@ public class RestServerSSLAuthITCase extends TestLogger { RestServerEndpointConfiguration restServerConfig = RestServerEndpointConfiguration.fromConfiguration(serverConfig); RestClientConfiguration restClientConfig = RestClientConfiguration.fromConfiguration(clientConfig); - RestfulGateway restfulGateway = TestingRestfulGateway.newBuilder().build(); + RestfulGateway restfulGateway = new TestingRestfulGateway.Builder().build(); RestServerEndpointITCase.TestVersionHandler testVersionHandler = new RestServerEndpointITCase.TestVersionHandler( () -> CompletableFuture.completedFuture(restfulGateway), RpcUtils.INF_TIMEOUT); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/AbstractHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/AbstractHandlerTest.java index becbba7659..f07ee37587 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/AbstractHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/AbstractHandlerTest.java @@ -71,7 +71,7 @@ public class AbstractHandlerTest extends TestLogger { final Path file = dir.resolve("file"); Files.createFile(file); - RestfulGateway mockRestfulGateway = TestingRestfulGateway.newBuilder() + RestfulGateway mockRestfulGateway = new TestingRestfulGateway.Builder() .build(); final GatewayRetriever mockGatewayRetriever = () -> diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandlerTest.java index d9f15601d3..a626d340ac 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandlerTest.java @@ -65,7 +65,7 @@ public class JobExecutionResultHandlerTest extends TestLogger { @Before public void setUp() throws Exception { - final TestingRestfulGateway testingRestfulGateway = TestingRestfulGateway.newBuilder().build(); + final TestingRestfulGateway testingRestfulGateway = new TestingRestfulGateway.Builder().build(); jobExecutionResultHandler = new JobExecutionResultHandler( () -> CompletableFuture.completedFuture(testingRestfulGateway), @@ -81,7 +81,7 @@ public class JobExecutionResultHandlerTest extends TestLogger { @Test public void testResultInProgress() throws Exception { - final TestingRestfulGateway testingRestfulGateway = TestingRestfulGateway.newBuilder() + final TestingRestfulGateway testingRestfulGateway = new TestingRestfulGateway.Builder() .setRequestJobStatusFunction( jobId -> CompletableFuture.completedFuture(JobStatus.RUNNING)) .build(); @@ -103,7 +103,7 @@ public class JobExecutionResultHandlerTest extends TestLogger { .setState(jobStatus) .build(); - final TestingRestfulGateway testingRestfulGateway = TestingRestfulGateway.newBuilder() + final TestingRestfulGateway testingRestfulGateway = new TestingRestfulGateway.Builder() .setRequestJobStatusFunction( jobId -> { assertThat(jobId, equalTo(TEST_JOB_ID)); @@ -129,7 +129,7 @@ public class JobExecutionResultHandlerTest extends TestLogger { @Test public void testPropagateFlinkJobNotFoundExceptionAsRestHandlerException() throws Exception { - final TestingRestfulGateway testingRestfulGateway = TestingRestfulGateway.newBuilder() + final TestingRestfulGateway testingRestfulGateway = new TestingRestfulGateway.Builder() .setRequestJobStatusFunction( jobId -> FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)) ) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobVertexBackPressureHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobVertexBackPressureHandlerTest.java index ec8be0fa5a..db0715a2b3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobVertexBackPressureHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobVertexBackPressureHandlerTest.java @@ -70,7 +70,7 @@ public class JobVertexBackPressureHandlerTest { @Before public void setUp() { - restfulGateway = TestingRestfulGateway.newBuilder().setRequestOperatorBackPressureStatsFunction( + restfulGateway = new TestingRestfulGateway.Builder().setRequestOperatorBackPressureStatsFunction( (jobId, jobVertexId) -> { if (jobId.equals(TEST_JOB_ID_BACK_PRESSURE_STATS_AVAILABLE)) { return CompletableFuture.completedFuture(OperatorBackPressureStatsResponse.of(new OperatorBackPressureStats( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphCacheTest.java index 545b34a335..0ccfaaa58e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphCacheTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/ExecutionGraphCacheTest.java @@ -168,7 +168,7 @@ public class ExecutionGraphCacheTest extends TestLogger { final ArchivedExecutionGraph expectedExecutionGraph2 = new ArchivedExecutionGraphBuilder().build(); final AtomicInteger requestJobCalls = new AtomicInteger(0); - final TestingRestfulGateway restfulGateway = TestingRestfulGateway.newBuilder() + final TestingRestfulGateway restfulGateway = new TestingRestfulGateway.Builder() .setRequestJobFunction( jobId -> { requestJobCalls.incrementAndGet(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingDispatcherGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingDispatcherGateway.java index 24c79f3c4e..2ce36957a4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingDispatcherGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingDispatcherGateway.java @@ -135,7 +135,7 @@ public final class TestingDispatcherGateway extends TestingRestfulGateway implem /** * Builder for the {@link TestingDispatcherGateway}. */ - public static final class Builder extends TestingRestfulGateway.Builder { + public static final class Builder extends TestingRestfulGateway.AbstractBuilder { private Function> submitFunction; private Supplier>> listFunction; @@ -159,11 +159,16 @@ public final class TestingDispatcherGateway extends TestingRestfulGateway implem } @Override - public TestingRestfulGateway.Builder setRequestJobFunction(Function> requestJobFunction) { + public Builder setRequestJobFunction(Function> requestJobFunction) { // signature clash throw new UnsupportedOperationException("Use setRequestArchivedJobFunction() instead."); } + @Override + protected Builder self() { + return this; + } + public Builder setBlobServerPort(int blobServerPort) { this.blobServerPort = blobServerPort; return this; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java index 1a3c6d78a1..b309940025 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java @@ -197,14 +197,12 @@ public class TestingRestfulGateway implements RestfulGateway { return hostname; } - public static Builder newBuilder() { - return new Builder(); - } - /** - * Builder for the {@link TestingRestfulGateway}. + * Abstract builder class for {@link TestingRestfulGateway} and its subclasses. + * + * @param type of sub class */ - public static class Builder { + protected abstract static class AbstractBuilder { protected String address = LOCALHOST; protected String hostname = LOCALHOST; protected Function> cancelJobFunction; @@ -220,7 +218,7 @@ public class TestingRestfulGateway implements RestfulGateway { protected BiFunction> triggerSavepointFunction; protected BiFunction> stopWithSavepointFunction; - public Builder() { + protected AbstractBuilder() { cancelJobFunction = DEFAULT_CANCEL_JOB_FUNCTION; requestJobFunction = DEFAULT_REQUEST_JOB_FUNCTION; requestJobResultFunction = DEFAULT_REQUEST_JOB_RESULT_FUNCTION; @@ -234,72 +232,88 @@ public class TestingRestfulGateway implements RestfulGateway { stopWithSavepointFunction = DEFAULT_STOP_WITH_SAVEPOINT_FUNCTION; } - public Builder setAddress(String address) { + public T setAddress(String address) { this.address = address; - return this; + return self(); } - public Builder setHostname(String hostname) { + public T setHostname(String hostname) { this.hostname = hostname; - return this; + return self(); } - public Builder setRequestJobFunction(Function> requestJobFunction) { + public T setRequestJobFunction(Function> requestJobFunction) { this.requestJobFunction = requestJobFunction; - return this; + return self(); } - public Builder setRequestJobResultFunction(Function> requestJobResultFunction) { + public T setRequestJobResultFunction(Function> requestJobResultFunction) { this.requestJobResultFunction = requestJobResultFunction; - return this; + return self(); } - public Builder setRequestJobStatusFunction(Function> requestJobStatusFunction) { + public T setRequestJobStatusFunction(Function> requestJobStatusFunction) { this.requestJobStatusFunction = requestJobStatusFunction; - return this; + return self(); } - public Builder setRequestMultipleJobDetailsSupplier(Supplier> requestMultipleJobDetailsSupplier) { + public T setRequestMultipleJobDetailsSupplier(Supplier> requestMultipleJobDetailsSupplier) { this.requestMultipleJobDetailsSupplier = requestMultipleJobDetailsSupplier; - return this; + return self(); } - public Builder setRequestClusterOverviewSupplier(Supplier> requestClusterOverviewSupplier) { + public T setRequestClusterOverviewSupplier(Supplier> requestClusterOverviewSupplier) { this.requestClusterOverviewSupplier = requestClusterOverviewSupplier; - return this; + return self(); } - public Builder setRequestMetricQueryServiceGatewaysSupplier(Supplier>> requestMetricQueryServiceGatewaysSupplier) { + public T setRequestMetricQueryServiceGatewaysSupplier(Supplier>> requestMetricQueryServiceGatewaysSupplier) { this.requestMetricQueryServiceGatewaysSupplier = requestMetricQueryServiceGatewaysSupplier; - return this; + return self(); } - public Builder setRequestTaskManagerMetricQueryServiceGatewaysSupplier(Supplier>>> requestTaskManagerMetricQueryServiceGatewaysSupplier) { + public T setRequestTaskManagerMetricQueryServiceGatewaysSupplier(Supplier>>> requestTaskManagerMetricQueryServiceGatewaysSupplier) { this.requestTaskManagerMetricQueryServiceGatewaysSupplier = requestTaskManagerMetricQueryServiceGatewaysSupplier; - return this; + return self(); } - public Builder setRequestOperatorBackPressureStatsFunction(BiFunction> requestOeratorBackPressureStatsFunction) { + public T setRequestOperatorBackPressureStatsFunction(BiFunction> requestOeratorBackPressureStatsFunction) { this.requestOperatorBackPressureStatsFunction = requestOeratorBackPressureStatsFunction; - return this; + return self(); } - public Builder setCancelJobFunction(Function> cancelJobFunction) { + public T setCancelJobFunction(Function> cancelJobFunction) { this.cancelJobFunction = cancelJobFunction; - return this; + return self(); } - public Builder setTriggerSavepointFunction(BiFunction> triggerSavepointFunction) { + public T setTriggerSavepointFunction(BiFunction> triggerSavepointFunction) { this.triggerSavepointFunction = triggerSavepointFunction; - return this; + return self(); } - public Builder setStopWithSavepointFunction(BiFunction> stopWithSavepointFunction) { + public T setStopWithSavepointFunction(BiFunction> stopWithSavepointFunction) { this.stopWithSavepointFunction = stopWithSavepointFunction; + return self(); + } + + protected abstract T self(); + + public abstract TestingRestfulGateway build(); + } + + /** + * Builder for the {@link TestingRestfulGateway}. + */ + public static class Builder extends AbstractBuilder { + + @Override + protected Builder self() { return this; } - public TestingRestfulGateway build() { + @Override + public TestingRestfulGateway build() { return new TestingRestfulGateway( address, hostname, -- Gitee From eb1bf7dcaf2307e0c040d12de9892f6fd304b23b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 13 Sep 2019 23:55:17 +0200 Subject: [PATCH 174/268] [hotfix] Fix checkstyle violations in RpcEndpoint --- .../src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index 7c7a4c188c..bc0da64cf6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -327,7 +327,6 @@ public abstract class RpcEndpoint implements RpcGateway, AutoCloseableAsync { // Asynchronous executions // ------------------------------------------------------------------------ - /** * Execute the runnable in the main thread of the underlying RPC endpoint. * -- Gitee From aa09f11c8d0f3782e2a3c6c15b01bb7935d6fa5b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 17:56:08 +0200 Subject: [PATCH 175/268] [hotfix] Add JobGraphStoreFactory --- .../jobmanager/JobGraphStoreFactory.java | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStoreFactory.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStoreFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStoreFactory.java new file mode 100644 index 0000000000..27110025b8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStoreFactory.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.runtime.jobmanager; + +/** + * Factory for {@link JobGraphStore}. + */ +public interface JobGraphStoreFactory { + + /** + * Creates a {@link JobGraphStore}. + * + * @return a {@link JobGraphStore} instance + */ + JobGraphStore create(); +} -- Gitee From 707975e518a563ba201a34f6a9da891c47327112 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 15 Sep 2019 16:32:32 +0200 Subject: [PATCH 176/268] [hotfix] Add TriFunctionWithException --- .../function/TriFunctionWithException.java | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) create mode 100644 flink-core/src/main/java/org/apache/flink/util/function/TriFunctionWithException.java diff --git a/flink-core/src/main/java/org/apache/flink/util/function/TriFunctionWithException.java b/flink-core/src/main/java/org/apache/flink/util/function/TriFunctionWithException.java new file mode 100644 index 0000000000..d845fb7b83 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/util/function/TriFunctionWithException.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.util.function; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.ExceptionUtils; + +import java.util.function.BiFunction; + +/** + * Function which takes three arguments. + * + * @param type of the first argument + * @param type of the second argument + * @param type of the third argument + * @param type of the return value + * @param type of the thrown exception + */ +@PublicEvolving +@FunctionalInterface +public interface TriFunctionWithException { + + /** + * Applies this function to the given arguments. + * + * @param s the first function argument + * @param t the second function argument + * @param u the third function argument + * @return the function result + * @throws E if it fails + */ + R apply(S s, T t, U u) throws E; + + /** + * Convert at {@link TriFunctionWithException} into a {@link TriFunction}. + * + * @param triFunctionWithException function with exception to convert into a function + * @param first input type + * @param second input type + * @param third input type + * @param output type + * @return {@link BiFunction} which throws all checked exception as an unchecked exception. + */ + static TriFunction unchecked(TriFunctionWithException triFunctionWithException) { + return (A a, B b, C c) -> { + try { + return triFunctionWithException.apply(a, b, c); + } catch (Throwable t) { + ExceptionUtils.rethrow(t); + // we need this to appease the compiler :-( + return null; + } + }; + } +} -- Gitee From 467736ab5d84ee168e6700a779bacb4351e506c5 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 17 Sep 2019 18:20:00 +0200 Subject: [PATCH 177/268] [hotfix] Let AbstractMetricGroupTest extend TestLogger --- .../flink/runtime/metrics/groups/AbstractMetricGroupTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 e31c3792be..790e85f6e2 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 @@ -34,6 +34,7 @@ import org.apache.flink.runtime.metrics.ReporterSetup; import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; import org.apache.flink.runtime.metrics.scope.ScopeFormats; import org.apache.flink.runtime.metrics.util.TestReporter; +import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -45,7 +46,7 @@ import static org.junit.Assert.assertTrue; /** * Tests for the {@link AbstractMetricGroup}. */ -public class AbstractMetricGroupTest { +public class AbstractMetricGroupTest extends TestLogger { /** * Verifies that no {@link NullPointerException} is thrown when {@link AbstractMetricGroup#getAllVariables()} is * called and the parent is null. -- Gitee From cda6dc0c44239aa7a36105988328de5744aea125 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 22:24:55 +0200 Subject: [PATCH 178/268] [FLINK-14307] Extract JobGraphWriter from JobGraphStore This closes #9830. --- .../runtime/jobmanager/JobGraphStore.java | 25 +--------- .../runtime/jobmanager/JobGraphWriter.java | 50 +++++++++++++++++++ 2 files changed, 51 insertions(+), 24 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphWriter.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStore.java index 634ec3790e..61cf8c6f74 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStore.java @@ -28,7 +28,7 @@ import java.util.Collection; /** * {@link JobGraph} instances for recovery. */ -public interface JobGraphStore { +public interface JobGraphStore extends JobGraphWriter { /** * Starts the {@link JobGraphStore} service. @@ -47,29 +47,6 @@ public interface JobGraphStore { @Nullable JobGraph recoverJobGraph(JobID jobId) throws Exception; - /** - * Adds the {@link JobGraph} instance. - * - *